diff --git a/core/src/main/java/org/apache/druid/indexer/TaskLocation.java b/core/src/main/java/org/apache/druid/indexer/TaskLocation.java index fef9b534f75e..1cc83db4d82f 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskLocation.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskLocation.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.net.HostAndPort; import org.apache.druid.java.util.common.IAE; import javax.annotation.Nullable; @@ -42,6 +43,11 @@ public static TaskLocation create(String host, int port, int tlsPort) return new TaskLocation(host, port, tlsPort); } + public static TaskLocation create(String host, int port, int tlsPort, boolean isTls) + { + return isTls ? new TaskLocation(host, -1, tlsPort) : new TaskLocation(host, port, -1); + } + public static TaskLocation unknown() { return TaskLocation.UNKNOWN; @@ -127,4 +133,12 @@ public int hashCode() { return Objects.hash(host, port, tlsPort); } + + public HostAndPort toHostAndPort() + { + if (tlsPort >= 0) { + return HostAndPort.fromParts(host, tlsPort); + } + return HostAndPort.fromParts(host, port); + } } diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatus.java index d3290bd88b1b..714c08be65d4 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatus.java @@ -66,6 +66,11 @@ public static TaskStatus failure(String taskId, String errorMsg) return new TaskStatus(taskId, TaskState.FAILED, -1, errorMsg, null); } + public static TaskStatus success(String taskId, TaskLocation location) + { + return new TaskStatus(taskId, TaskState.SUCCESS, -1, null, location); + } + /** * This method is deprecated for production because it does not handle the error message of failed task status properly. * Use {@link #success(String)} or {@link #failure(String, String)} instead. diff --git a/core/src/test/java/org/apache/druid/indexer/TaskStatusTest.java b/core/src/test/java/org/apache/druid/indexer/TaskStatusTest.java index 92651bca8154..939d9f04d372 100644 --- a/core/src/test/java/org/apache/druid/indexer/TaskStatusTest.java +++ b/core/src/test/java/org/apache/druid/indexer/TaskStatusTest.java @@ -58,5 +58,10 @@ public void testSerde() throws IOException null ); Assert.assertEquals(statusNoLocation, mapper.readValue(jsonNoLocation, TaskStatus.class)); + + TaskStatus success = TaskStatus.success("forkTaskID", new TaskLocation("localhost", 0, 1)); + Assert.assertEquals(success.getLocation().getHost(), "localhost"); + Assert.assertEquals(success.getLocation().getPort(), 0); + Assert.assertEquals(success.getLocation().getTlsPort(), 1); } } diff --git a/distribution/docker/Dockerfile b/distribution/docker/Dockerfile index 5b55ec1be6cd..5f01c3dc09f4 100644 --- a/distribution/docker/Dockerfile +++ b/distribution/docker/Dockerfile @@ -81,6 +81,7 @@ RUN addgroup -S -g 1000 druid \ COPY --chown=druid:druid --from=builder /opt /opt COPY distribution/docker/druid.sh /druid.sh +COPY distribution/docker/peon.sh /peon.sh # create necessary directories which could be mounted as volume # /opt/druid/var is used to keep individual files(e.g. log) of each Druid service diff --git a/distribution/docker/peon.sh b/distribution/docker/peon.sh new file mode 100755 index 000000000000..580db127e3b9 --- /dev/null +++ b/distribution/docker/peon.sh @@ -0,0 +1,153 @@ +#!/bin/sh + +# +# 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. +# + +# NOTE: this is a 'run' script for the stock tarball +# It takes 1 required argument (the name of the service, +# e.g. 'broker', 'historical' etc). Any additional arguments +# are passed to that service. +# +# It accepts 'JAVA_OPTS' as an environment variable +# +# Additional env vars: +# - DRUID_LOG4J -- set the entire log4j.xml verbatim +# - DRUID_LOG_LEVEL -- override the default log level in default log4j +# - DRUID_XMX -- set Java Xmx +# - DRUID_XMS -- set Java Xms +# - DRUID_MAXNEWSIZE -- set Java max new size +# - DRUID_NEWSIZE -- set Java new size +# - DRUID_MAXDIRECTMEMORYSIZE -- set Java max direct memory size +# +# - DRUID_CONFIG_COMMON -- full path to a file for druid 'common' properties +# - DRUID_CONFIG_${service} -- full path to a file for druid 'service' properties + +set -e +SERVICE="overlord" + +echo "$(date -Is) startup service $SERVICE" + +# We put all the config in /tmp/conf to allow for a +# read-only root filesystem +mkdir -p /tmp/conf/ +test -d /tmp/conf/druid && rm -r /tmp/conf/druid +cp -r /opt/druid/conf/druid /tmp/conf/druid + +getConfPath() { + cluster_conf_base=/tmp/conf/druid/cluster + case "$1" in + _common) echo $cluster_conf_base/_common ;; + historical) echo $cluster_conf_base/data/historical ;; + middleManager) echo $cluster_conf_base/data/middleManager ;; + indexer) echo $cluster_conf_base/data/indexer ;; + coordinator | overlord) echo $cluster_conf_base/master/coordinator-overlord ;; + broker) echo $cluster_conf_base/query/broker ;; + router) echo $cluster_conf_base/query/router ;; + *) echo $cluster_conf_base/misc/$1 ;; + esac +} +COMMON_CONF_DIR=$(getConfPath _common) +SERVICE_CONF_DIR=$(getConfPath ${SERVICE}) + +# Delete the old key (if existing) and append new key=value +setKey() { + service="$1" + key="$2" + value="$3" + service_conf=$(getConfPath $service)/runtime.properties + # Delete from all + sed -ri "/$key=/d" $COMMON_CONF_DIR/common.runtime.properties + [ -f $service_conf ] && sed -ri "/$key=/d" $service_conf + [ -f $service_conf ] && echo -e "\n$key=$value" >>$service_conf + [ -f $service_conf ] || echo -e "\n$key=$value" >>$COMMON_CONF_DIR/common.runtime.properties + + echo "Setting $key=$value in $service_conf" +} + +setJavaKey() { + service="$1" + key=$2 + value=$3 + file=$(getConfPath $service)/jvm.config + sed -ri "/$key/d" $file + echo $value >> $file +} + +## Setup host names +if [ -n "${ZOOKEEPER}" ]; +then + setKey _common druid.zk.service.host "${ZOOKEEPER}" +fi + +DRUID_SET_HOST=${DRUID_SET_HOST:-1} +if [ "${DRUID_SET_HOST}" = "1" ] +then + setKey $SERVICE druid.host $(ip r get 1 | awk '{print $7;exit}') +fi + +env | grep ^druid_ | while read evar; +do + # Can't use IFS='=' to parse since var might have = in it (e.g. password) + val=$(echo "$evar" | sed -e 's?[^=]*=??') + var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?_?.?g') + setKey $SERVICE "$var" "$val" +done + +env |grep ^s3service | while read evar +do + val=$(echo "$evar" | sed -e 's?[^=]*=??') + var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?_?.?' -e 's?_?-?g') + echo "$var=$val" >>$COMMON_CONF_DIR/jets3t.properties +done + +# This is to allow configuration via a Kubernetes configMap without +# e.g. using subPath (you can also mount the configMap on /tmp/conf/druid) +if [ -n "$DRUID_CONFIG_COMMON" ] +then + cp -f "$DRUID_CONFIG_COMMON" $COMMON_CONF_DIR/common.runtime.properties +fi + +SCONFIG=$(printf "%s_%s" DRUID_CONFIG ${SERVICE}) +SCONFIG=$(eval echo \$$(echo $SCONFIG)) + +if [ -n "${SCONFIG}" ] +then + cp -f "${SCONFIG}" $SERVICE_CONF_DIR/runtime.properties +fi + +if [ -n "$DRUID_LOG_LEVEL" ] +then + sed -ri 's/"info"/"'$DRUID_LOG_LEVEL'"/g' $COMMON_CONF_DIR/log4j2.xml +fi + +if [ -n "$DRUID_LOG4J" ] +then + echo "$DRUID_LOG4J" >$COMMON_CONF_DIR/log4j2.xml +fi + +DRUID_DIRS_TO_CREATE=${DRUID_DIRS_TO_CREATE-'var/tmp var/druid/segments var/druid/indexing-logs var/druid/task var/druid/hadoop-tmp var/druid/segment-cache'} +if [ -n "${DRUID_DIRS_TO_CREATE}" ] +then + mkdir -p ${DRUID_DIRS_TO_CREATE} +fi + +# take the ${TASK_JSON} environment variable and base64 decode, unzip and throw it in ${TASK_DIR}/task.json +mkdir -p ${TASK_DIR}; echo ${TASK_JSON} | base64 -d | gzip -d > ${TASK_DIR}/task.json; + +exec java ${JAVA_OPTS} -cp $COMMON_CONF_DIR:$SERVICE_CONF_DIR:lib/*: org.apache.druid.cli.Main internal peon $@ diff --git a/distribution/pom.xml b/distribution/pom.xml index 729d6f7bcda9..ee21a634fab4 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -595,6 +595,8 @@ -c org.apache.druid.extensions.contrib:kafka-emitter -c + org.apache.druid.extensions.contrib:druid-kubernetes-overlord-extensions + -c org.apache.druid.extensions.contrib:materialized-view-maintenance -c org.apache.druid.extensions.contrib:materialized-view-selection diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md new file mode 100644 index 000000000000..9201d553d8f8 --- /dev/null +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -0,0 +1,96 @@ +--- +id: k8s-jobs +title: "MM-less Druid in K8s" +--- + + + +Consider this an [EXPERIMENTAL](../experimental.md) feature mostly because it has not been tested yet on a wide variety of long-running Druid clusters. + +Apache Druid Extension to enable using Kubernetes for launching and managing tasks instead of the Middle Managers. This extension allows you to launch tasks as K8s jobs removing the need for your middle manager. + +## How it works + +It takes the podSpec of your `Overlord`pod and creates a kubernetes job from this podSpec. Thus if you have sidecars such as splunk, hubble, istio it can optionally launch a task as a k8s job. All jobs are natively restorable, they are decopled from the druid deployment, thus restarting pods or doing upgrades has no affect on tasks in flight. They will continue to run and when the overlord comes back up it will start tracking them again. + +## Configuration + +To use this extension please make sure to [include](../extensions.md#loading-extensions)`druid-kubernetes-overlord-extensions` in the extensions load list for your overlord process. + +The extension uses the task queue to limit how many concurrent tasks (k8s jobs) are in flight so it is required you have a reasonable value for `druid.indexer.queue.maxSize`. Additionally set the variable `druid.indexer.runner.namespace` to the namespace in which you are running druid. + +Other configurations required are: +`druid.indexer.runner.type: k8s` and `druid.indexer.task.enableTaskLevelLogPush: true` + +You can add optional labels to your k8s jobs / pods if you need them by using the following configuration: +`druid.indexer.runner.labels: '{"key":"value"}'` + +All other configurations you had for the middle manager tasks must be moved under the overlord with one caveat, you must specify javaOpts as an array: +`druid.indexer.runner.javaOptsArray`, `druid.indexer.runner.javaOpts` is no longer supported. + +If you are running without a middle manager you need to also use `druid.processing.intermediaryData.storage.type=deepstore` + +Additional Configuration + +### Properties +|Property|Possible Values|Description|Default|required| +|--------|---------------|-----------|-------|--------| +|`druid.indexer.runner.debugJobs`|`boolean`|Clean up k8s jobs after tasks complete.|False|No| +|`druid.indexer.runner.sidecarSupport`|`boolean`|If your overlord pod has sidecars, this will attempt to start the task with the same sidecars as the overlord pod.|False|No| +|`druid.indexer.runner.kubexitImage`|`String`|Used kubexit project to help shutdown sidecars when the main pod completes. Otherwise jobs with sidecars never terminate.|karlkfi/kubexit:latest|No| +|`druid.indexer.runner.maxTaskDuration`|`Duration`|Max time a task is allowed to run for before getting killed|4H|No| +|`druid.indexer.runner.taskCleanupDelay`|`Duration`|How long do jobs stay around before getting reaped from k8s|2D|No| +|`druid.indexer.runner.taskCleanupInterval`|`Duration`|How often to check for jobs to be reaped|10m|No| +|`druid.indexer.runner.k8sjobLaunchTimeout`|`Duration`|How long to wait to launch a k8s task before marking it as failed, on a resource constrained cluster it may take some time.|1H|No| +|`druid.indexer.runner.javaOptsArray`|`Duration`|java opts for the task.|-Xmx1g|No| +|`druid.indexer.runner.graceTerminationPeriodSeconds`|`Long`|Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods.|30s (k8s default)|No| + +### Gotchas + +- You must have in your role the abiliity to launch jobs. +- All Druid Pods belonging to one Druid cluster must be inside same kubernetes namespace. + +``` +apiVersion: rbac.authorization.k8s.io/v1 +kind: Role +metadata: + name: druid-cluster +rules: +- apiGroups: + - "" + resources: + - pods + - configmaps + - jobs + verbs: + - '*' +--- +kind: RoleBinding +apiVersion: rbac.authorization.k8s.io/v1 +metadata: + name: druid-cluster +subjects: +- kind: ServiceAccount + name: default +roleRef: + kind: Role + name: druid-cluster + apiGroup: rbac.authorization.k8s.io +``` diff --git a/docs/development/extensions.md b/docs/development/extensions.md index 4c2fc3977089..5957ac51f30e 100644 --- a/docs/development/extensions.md +++ b/docs/development/extensions.md @@ -97,6 +97,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati |druid-tdigestsketch|Support for approximate sketch aggregators based on [T-Digest](https://github.com/tdunning/t-digest)|[link](../development/extensions-contrib/tdigestsketch-quantiles.md)| |gce-extensions|GCE Extensions|[link](../development/extensions-contrib/gce-extensions.md)| |prometheus-emitter|Exposes [Druid metrics](../operations/metrics.md) for Prometheus server collection (https://prometheus.io/)|[link](./extensions-contrib/prometheus.md)| +|kubernetes-overlord-extensions|Support for launching tasks in k8s, no more Middle Managers|[link](../development/extensions-contrib/k8s-jobs.md)| ## Promoting community extensions to core extensions diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml new file mode 100644 index 000000000000..047d853b2b33 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml @@ -0,0 +1,186 @@ + + + + + 4.0.0 + + org.apache.druid.extensions.contrib + druid-kubernetes-overlord-extensions + druid-kubernetes-overlord-extensions + druid-kubernetes-overlord-extensions + + + org.apache.druid + druid + 25.0.0-SNAPSHOT + ../../pom.xml + + + + + + com.squareup.okhttp3 + okhttp + 3.14.9 + + + com.squareup.okhttp3 + logging-interceptor + 3.14.9 + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + com.squareup.okhttp3 + okhttp + + + com.squareup.okhttp3 + logging-interceptor + + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + + io.fabric8 + kubernetes-client + 5.12.2 + + + + + junit + junit + test + + + org.easymock + easymock + test + + + io.fabric8 + kubernetes-server-mock + 5.12.2 + test + + + + + com.google.code.findbugs + jsr305 + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + joda-time + joda-time + provided + + + com.google.guava + guava + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + org.mockito + mockito-all + 1.9.5 + test + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + + + \ No newline at end of file diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordConfig.java new file mode 100644 index 000000000000..55f6bd95bb32 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordConfig.java @@ -0,0 +1,24 @@ +/* + * 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; + +public class K8sOverlordConfig +{ +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java new file mode 100644 index 000000000000..dc1fca059e8f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.multibindings.MapBinder; +import org.apache.druid.guice.Binders; +import org.apache.druid.guice.IndexingServiceModuleHelper; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.PolyBind; +import org.apache.druid.indexing.common.config.FileTaskLogsConfig; +import org.apache.druid.indexing.common.tasklogs.FileTaskLogs; +import org.apache.druid.indexing.overlord.TaskRunnerFactory; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.tasklogs.NoopTaskLogs; +import org.apache.druid.tasklogs.TaskLogKiller; +import org.apache.druid.tasklogs.TaskLogPusher; +import org.apache.druid.tasklogs.TaskLogs; + +import java.util.Collections; +import java.util.List; + +public class K8sOverlordModule implements DruidModule +{ + + @Override + public void configure(Binder binder) + { + // druid.indexer.runner.type=k8s + JsonConfigProvider.bind(binder, IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX, KubernetesTaskRunnerConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); + PolyBind.createChoice( + binder, + "druid.indexer.runner.type", + Key.get(TaskRunnerFactory.class), + Key.get(KubernetesTaskRunnerFactory.class) + ); + final MapBinder biddy = PolyBind.optionBinder( + binder, + Key.get(TaskRunnerFactory.class) + ); + + biddy.addBinding(KubernetesTaskRunnerFactory.TYPE_NAME) + .to(KubernetesTaskRunnerFactory.class) + .in(LazySingleton.class); + binder.bind(KubernetesTaskRunnerFactory.class).in(LazySingleton.class); + configureTaskLogs(binder); + } + + private void configureTaskLogs(Binder binder) + { + PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(FileTaskLogs.class)); + JsonConfigProvider.bind(binder, "druid.indexer.logs", FileTaskLogsConfig.class); + + final MapBinder taskLogBinder = Binders.taskLogsBinder(binder); + taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class); + taskLogBinder.addBinding("file").to(FileTaskLogs.class).in(LazySingleton.class); + binder.bind(NoopTaskLogs.class).in(LazySingleton.class); + binder.bind(FileTaskLogs.class).in(LazySingleton.class); + + binder.bind(TaskLogPusher.class).to(TaskLogs.class); + binder.bind(TaskLogKiller.class).to(TaskLogs.class); + } + + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java new file mode 100644 index 000000000000..91a83bc099da --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -0,0 +1,571 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.netty.util.SuppressForbidden; +import org.apache.commons.io.FileUtils; +import org.apache.druid.indexer.RunnerTaskState; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.ForkingTaskRunner; +import org.apache.druid.indexing.overlord.QuotableWhiteSpaceSplitter; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerUtils; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; +import org.apache.druid.indexing.overlord.config.ForkingTaskRunnerConfig; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.common.JobResponse; +import org.apache.druid.k8s.overlord.common.K8sTaskId; +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.common.TaskAdapter; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogPusher; +import org.apache.druid.tasklogs.TaskLogStreamer; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Runs tasks as k8s jobs using the "internal peon" verb. + * One additional feature of this class is that kubernetes is the source of truth, so if you launch a task + * shutdown druid, bring up druid, the task will keep running and the state will be updated when the cluster + * comes back. Thus while no tasks are technically restorable, all tasks once launched will run in isolation to the + * extent possible without requiring the overlord consistently up during their lifetime. + */ + +public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner +{ + + private static final EmittingLogger log = new EmittingLogger(KubernetesTaskRunner.class); + private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); + + // to cleanup old jobs that might not have been deleted. + private final ScheduledExecutorService cleanupExecutor; + + protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + private final TaskConfig taskConfig; + private final StartupLoggingConfig startupLoggingConfig; + private final TaskAdapter adapter; + + private final KubernetesTaskRunnerConfig k8sConfig; + private final TaskQueueConfig taskQueueConfig; + private final TaskLogPusher taskLogPusher; + private final ListeningExecutorService exec; + private final KubernetesPeonClient client; + private final DruidNode node; + + + public KubernetesTaskRunner( + TaskConfig taskConfig, + StartupLoggingConfig startupLoggingConfig, + TaskAdapter adapter, + KubernetesTaskRunnerConfig k8sConfig, + TaskQueueConfig taskQueueConfig, + TaskLogPusher taskLogPusher, + KubernetesPeonClient client, + DruidNode node + ) + { + this.taskConfig = taskConfig; + this.startupLoggingConfig = startupLoggingConfig; + this.adapter = adapter; + this.k8sConfig = k8sConfig; + this.taskQueueConfig = taskQueueConfig; + this.taskLogPusher = taskLogPusher; + this.client = client; + this.node = node; + this.cleanupExecutor = Executors.newScheduledThreadPool(1); + this.exec = MoreExecutors.listeningDecorator( + Execs.multiThreaded(taskQueueConfig.getMaxSize(), "k8s-task-runner-%d") + ); + Preconditions.checkArgument( + taskQueueConfig.getMaxSize() < Integer.MAX_VALUE, + "The task queue bounds how many concurrent k8s tasks you can have, set it to a resonable value" + ); + } + + + @Override + public Optional streamTaskLog(String taskid, long offset) + { + Optional peonLogs = client.getPeonLogs(new K8sTaskId(taskid)); + if (!peonLogs.isPresent()) { + return Optional.absent(); + } + return Optional.of(peonLogs.get()); + } + + @Override + public ListenableFuture run(Task task) + { + synchronized (tasks) { + tasks.computeIfAbsent( + task.getId(), k -> new K8sWorkItem( + task, + exec.submit(() -> { + K8sTaskId k8sTaskId = new K8sTaskId(task); + try { + JobResponse completedPhase; + Optional existingJob = client.jobExists(k8sTaskId); + if (!existingJob.isPresent()) { + PeonCommandContext context = new PeonCommandContext( + generateCommand(task), + javaOpts(task), + taskConfig.getTaskDir(task.getId()), + node.isEnableTlsPort() + ); + Job job = adapter.fromTask(task, context); + log.info("Job created %s and ready to launch", k8sTaskId); + Pod peonPod = client.launchJobAndWaitForStart( + job, + KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.k8sjobLaunchTimeout), + TimeUnit.MILLISECONDS + ); + log.info("Job %s launched in k8s", k8sTaskId); + completedPhase = monitorJob(peonPod, task, k8sTaskId); + } else { + Job job = existingJob.get(); + if (job.getStatus().getActive() == null) { + if (job.getStatus().getSucceeded() != null) { + completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED); + } else { + completedPhase = new JobResponse(job, PeonPhase.FAILED); + } + } else { + // the job is active lets monitor it + completedPhase = monitorJob(task, k8sTaskId); + } + } + TaskStatus status; + if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) { + status = TaskStatus.success(task.getId(), completedPhase.getLocation()); + } else { + status = TaskStatus.failure( + task.getId(), + "Task failed %s: " + k8sTaskId + ); + } + if (completedPhase.getJobDuration().isPresent()) { + status = status.withDuration(completedPhase.getJobDuration().get()); + } + updateStatus(task, status); + return status; + } + catch (Exception e) { + log.error(e, "Error with task: %s", k8sTaskId); + throw e; + } + finally { + // publish task logs + Path log = Files.createTempFile(task.getId(), "log"); + try { + FileUtils.write( + log.toFile(), + client.getJobLogs(new K8sTaskId(task.getId())), + StandardCharsets.UTF_8 + ); + taskLogPusher.pushTaskLog(task.getId(), log.toFile()); + } + finally { + Files.deleteIfExists(log); + } + client.cleanUpJob(new K8sTaskId(task.getId())); + synchronized (tasks) { + tasks.remove(task.getId()); + } + } + }) + )); + return tasks.get(task.getId()).getResult(); + } + } + + JobResponse monitorJob(Task task, K8sTaskId k8sTaskId) + { + return monitorJob(client.getMainJobPod(k8sTaskId), task, k8sTaskId); + } + + JobResponse monitorJob(Pod peonPod, Task task, K8sTaskId k8sTaskId) + { + if (peonPod == null) { + throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId); + } + log.info("monitorJob: Enable TLS Port is " + node.isEnableTlsPort()); + log.info("Pod Host IP " + peonPod.getStatus().getPodIP()); + TaskLocation location = TaskLocation.create( + peonPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT, + node.isEnableTlsPort() + ); + client.waitForProcessToStart(location, 10, TimeUnit.MINUTES); + + updateLocation(task, location); + updateStatus(task, TaskStatus.running(task.getId())); + + JobResponse response = client.waitForJobCompletion( + k8sTaskId, + KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration), + TimeUnit.MILLISECONDS + ); + response.setLocation(location); + return response; + } + + @VisibleForTesting + void updateStatus(Task task, TaskStatus status) + { + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); + } + + @VisibleForTesting + void updateLocation(Task task, TaskLocation location) + { + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), location); + } + + @Override + public void shutdown(String taskid, String reason) + { + client.cleanUpJob(new K8sTaskId(taskid)); + } + + + @Override + public Optional streamTaskReports(String taskid) + { + return Optional.absent(); + } + + @Override + public List>> restore() + { + return ImmutableList.of(); + } + + @Override + public void start() + { + log.info("Starting cleanup executor for jobs older than 1 day...."); + cleanupExecutor.scheduleAtFixedRate( + () -> + client.cleanCompletedJobsOlderThan( + KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupDelay), + TimeUnit.MILLISECONDS + ), + 1, + KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupInterval), + TimeUnit.MILLISECONDS + ); + log.info("Started cleanup executor for jobs older than 1 day...."); + } + + + @Override + public void stop() + { + log.info("Stopping..."); + cleanupExecutor.shutdownNow(); + log.info("Stopped..."); + } + + @Override + public Map getTotalTaskSlotCount() + { + return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize()); + } + + private List javaOpts(Task task) + { + final List javaOpts = new ArrayList<>(); + Iterables.addAll(javaOpts, k8sConfig.javaOptsArray); + + // Override task specific javaOpts + Object taskJavaOpts = task.getContextValue( + ForkingTaskRunnerConfig.JAVA_OPTS_PROPERTY + ); + if (taskJavaOpts != null) { + Iterables.addAll( + javaOpts, + new QuotableWhiteSpaceSplitter((String) taskJavaOpts) + ); + } + + javaOpts.add(StringUtils.format("-Ddruid.port=%d", DruidK8sConstants.PORT)); + javaOpts.add(StringUtils.format("-Ddruid.plaintextPort=%d", DruidK8sConstants.PORT)); + javaOpts.add(StringUtils.format("-Ddruid.tlsPort=%d", node.isEnableTlsPort() ? DruidK8sConstants.TLS_PORT : -1)); + javaOpts.add(StringUtils.format( + "-Ddruid.task.executor.tlsPort=%d", + node.isEnableTlsPort() ? DruidK8sConstants.TLS_PORT : -1 + )); + javaOpts.add(StringUtils.format("-Ddruid.task.executor.enableTlsPort=%s", node.isEnableTlsPort()) + ); + return javaOpts; + } + + private List generateCommand(Task task) + { + final List command = new ArrayList<>(); + // must use deep storage shuffle now with k8s peon tasks. + + command.add("/peon.sh"); + command.add(taskConfig.getTaskDir(task.getId()).toString()); + command.add("1"); // the attemptId is always 1, we never run the task twice on the same pod. + + String nodeType = task.getNodeType(); + if (nodeType != null) { + command.add("--nodeType"); + command.add(nodeType); + } + + // If the task type is queryable, we need to load broadcast segments on the peon, used for + // join queries + if (task.supportsQueries()) { + command.add("--loadBroadcastSegments"); + command.add("true"); + } + log.info( + "Peon Command for K8s job: %s", + ForkingTaskRunner.getMaskedCommand(startupLoggingConfig.getMaskProperties(), command) + ); + return command; + } + + @Override + public Collection getKnownTasks() + { + List result = new ArrayList<>(); + for (Pod existingTask : client.listPeonPods()) { + try { + Task task = adapter.toTask(existingTask); + ListenableFuture future = run(task); + result.add(new K8sWorkItem(task, future, DateTimes.of(existingTask.getMetadata().getCreationTimestamp()))); + } + catch (IOException e) { + log.error("Error deserializing task from pod: " + existingTask.getMetadata().getName()); + } + + } + return result; + } + + + @Override + public Optional getScalingStats() + { + return Optional.absent(); + } + + @Override + public Map getIdleTaskSlotCount() + { + return Collections.emptyMap(); + } + + @Override + public Map getUsedTaskSlotCount() + { + return Collections.emptyMap(); + } + + @Override + public Map getLazyTaskSlotCount() + { + return Collections.emptyMap(); + } + + @Override + public Map getBlacklistedTaskSlotCount() + { + return Collections.emptyMap(); + } + + @Override + public boolean isK8sTaskRunner() + { + return true; + } + + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs != null && pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + log.info("Unregistered listener [%s]", listenerId); + return; + } + } + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + for (Pair pair : listeners) { + if (pair.lhs != null && pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + + final Pair listenerPair = Pair.of(listener, executor); + log.info("Registered listener [%s]", listener.getListenerId()); + listeners.add(listenerPair); + } + + @Override + @SuppressForbidden(reason = "Sets#newHashSet") + public Collection getRunningTasks() + { + List result = new ArrayList<>(); + for (Pod existingTask : client.listPeonPods(Sets.newHashSet(PeonPhase.RUNNING))) { + try { + Task task = adapter.toTask(existingTask); + ListenableFuture future = run(task); + result.add(new K8sWorkItem(task, future, DateTime.parse(existingTask.getMetadata().getCreationTimestamp()))); + } + catch (IOException e) { + log.error("Error deserializing task from pod: " + existingTask.getMetadata().getName()); + } + } + return result; + } + + @Override + public Collection getPendingTasks() + { + // the task queue limits concurrent tasks, we fire off to k8s right away + // thus nothing is really "pending" + return new ArrayList<>(); + } + + @Nullable + @Override + public RunnerTaskState getRunnerTaskState(String taskId) + { + Pod item = client.getMainJobPod(new K8sTaskId(taskId)); + if (item == null) { + return null; + } else { + PeonPhase phase = PeonPhase.getPhaseFor(item); + if (PeonPhase.PENDING.equals(phase)) { + return RunnerTaskState.PENDING; + } else if (PeonPhase.RUNNING.equals(phase)) { + return RunnerTaskState.RUNNING; + } else { + return RunnerTaskState.NONE; + } + } + } + + public class K8sWorkItem extends TaskRunnerWorkItem + { + private final Task task; + + private K8sWorkItem(Task task, ListenableFuture statusFuture) + { + super(task.getId(), statusFuture); + this.task = task; + } + + private K8sWorkItem(Task task, ListenableFuture statusFuture, DateTime createdTime) + { + super(task.getId(), statusFuture, createdTime, createdTime); + this.task = task; + } + + @Override + public TaskLocation getLocation() + { + K8sTaskId taskId = new K8sTaskId(task.getId()); + try { + Pod mainPod = client.getMainJobPod(new K8sTaskId(task.getId())); + boolean tlsEnabled = Boolean.parseBoolean( + mainPod.getMetadata() + .getAnnotations() + .getOrDefault(DruidK8sConstants.TLS_ENABLED, "false")); + return TaskLocation.create( + mainPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT, + tlsEnabled + ); + } + catch (Exception e) { + log.error(e, "Error getting task location for task %s", taskId); + return TaskLocation.unknown(); + } + } + + @Override + public String getTaskType() + { + return task.getType(); + } + + @Override + public String getDataSource() + { + return task.getDataSource(); + } + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java new file mode 100644 index 000000000000..25f198fb6a6f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; +import org.joda.time.Period; + +import javax.validation.constraints.NotNull; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class KubernetesTaskRunnerConfig +{ + + @JsonProperty + @NotNull + public String namespace; + + @JsonProperty + public boolean debugJobs = false; + + @JsonProperty + public boolean sidecarSupport = false; + + @JsonProperty + // for multi-container jobs, we need this image to shut down sidecars after the main container + // has completed + public String kubexitImage = "karlkfi/kubexit:v0.3.2"; + + // how much time to wait for preStop hooks to execute + // lower number speeds up pod termination time to release locks + // faster, defaults to your k8s setup, usually 30 seconds. + public Long graceTerminationPeriodSeconds = null; + + @JsonProperty + @NotNull + public Period maxTaskDuration = new Period("PT4H"); + + @JsonProperty + @NotNull + // how long to wait for the jobs to be cleaned up. + public Period taskCleanupDelay = new Period("P2D"); + + @JsonProperty + @NotNull + // interval for k8s job cleanup to run + public Period taskCleanupInterval = new Period("PT10m"); + + @JsonProperty + @NotNull + // how long to wait for the peon k8s job to launch + public Period k8sjobLaunchTimeout = new Period("PT1H"); + + @JsonProperty + @NotNull + public List javaOptsArray; + + @JsonProperty + @NotNull + public String classpath = System.getProperty("java.class.path"); + + @JsonProperty + @NotNull + public Map labels = new HashMap<>(); + + @JsonProperty + @NotNull + public Map annotations = new HashMap<>(); + + @JsonProperty + @NotNull + List allowedPrefixes = Lists.newArrayList( + "com.metamx", + "druid", + "org.apache.druid", + "user.timezone", + "file.encoding", + "java.io.tmpdir", + "hadoop" + ); + + public static long toMilliseconds(Period period) + { + return period.toStandardDuration().getMillis(); + } + +} 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 new file mode 100644 index 000000000000..b0d4b3fc4b91 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.overlord.TaskRunnerFactory; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; +import org.apache.druid.k8s.overlord.common.DruidKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.K8sTaskAdapter; +import org.apache.druid.k8s.overlord.common.MultiContainerTaskAdapter; +import org.apache.druid.k8s.overlord.common.SingleContainerTaskAdapter; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogPusher; + +public class KubernetesTaskRunnerFactory implements TaskRunnerFactory +{ + public static final String TYPE_NAME = "k8s"; + private final ObjectMapper smileMapper; + private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private final TaskConfig taskConfig; + private final StartupLoggingConfig startupLoggingConfig; + private final TaskQueueConfig taskQueueConfig; + private final TaskLogPusher taskLogPusher; + private final DruidNode druidNode; + + + @Inject + public KubernetesTaskRunnerFactory( + @Smile ObjectMapper smileMapper, + KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, + TaskConfig taskConfig, + StartupLoggingConfig startupLoggingConfig, + @JacksonInject TaskQueueConfig taskQueueConfig, + TaskLogPusher taskLogPusher, + @Self DruidNode druidNode + ) + { + + this.smileMapper = smileMapper; + this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig; + this.taskConfig = taskConfig; + this.startupLoggingConfig = startupLoggingConfig; + this.taskQueueConfig = taskQueueConfig; + this.taskLogPusher = taskLogPusher; + this.druidNode = druidNode; + } + + @Override + public KubernetesTaskRunner build() + { + DruidKubernetesClient client = new DruidKubernetesClient(); + K8sTaskAdapter adapter; + if (kubernetesTaskRunnerConfig.sidecarSupport) { + adapter = new MultiContainerTaskAdapter(client, kubernetesTaskRunnerConfig, smileMapper); + } else { + adapter = new SingleContainerTaskAdapter(client, kubernetesTaskRunnerConfig, smileMapper); + } + + + return new KubernetesTaskRunner( + taskConfig, + startupLoggingConfig, + adapter, + kubernetesTaskRunnerConfig, + taskQueueConfig, + taskLogPusher, + new DruidKubernetesPeonClient(client, kubernetesTaskRunnerConfig.namespace, kubernetesTaskRunnerConfig.debugJobs), + druidNode + ); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/Base64Compression.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/Base64Compression.java new file mode 100644 index 000000000000..584efd486c8b --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/Base64Compression.java @@ -0,0 +1,51 @@ +/* + * 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.common; + +import com.google.api.client.util.Base64; +import org.apache.commons.io.IOUtils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +public class Base64Compression +{ + public static String compressBase64(String srcTxt) throws IOException + { + ByteArrayOutputStream rstBao = new ByteArrayOutputStream(); + try (GZIPOutputStream zos = new GZIPOutputStream(rstBao)) { + zos.write(srcTxt.getBytes(StandardCharsets.UTF_8)); + } + byte[] bytes = rstBao.toByteArray(); + return Base64.encodeBase64String(bytes); + } + + public static String decompressBase64(String zippedBase64Str) throws IOException + { + byte[] bytes = Base64.decodeBase64(zippedBase64Str); + try (GZIPInputStream zi = new GZIPInputStream(new ByteArrayInputStream(bytes))) { + return IOUtils.toString(zi, StandardCharsets.UTF_8); + } + } +} 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 new file mode 100644 index 000000000000..abdb74ee0c2a --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -0,0 +1,37 @@ +/* + * 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.common; + +import com.google.common.base.Predicate; + +public class DruidK8sConstants +{ + public static final String TASK_ID = "task.id"; + public static final int PORT = 8100; + public static final int TLS_PORT = 8091; + public static final String TLS_ENABLED = "tls.enabled"; + public static final String TASK_JSON_ENV = "TASK_JSON"; + public static final String TASK_DIR_ENV = "TASK_DIR"; + public static final String JAVA_OPTS = "JAVA_OPTS"; + public static final String DRUID_HOST_ENV = "druid_host"; + public static final String DRUID_HOSTNAME_ENV = "HOSTNAME"; + static final String LABEL_KEY = "druid.k8s.peons"; + 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/DruidKubernetesClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java new file mode 100644 index 000000000000..4ea3c690aad0 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.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.common; + +import io.fabric8.kubernetes.client.DefaultKubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClient; + +public class DruidKubernetesClient implements KubernetesClientApi +{ + @Override + public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException + { + try (KubernetesClient client = new DefaultKubernetesClient()) { + return executor.executeRequest(client); + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java new file mode 100644 index 000000000000..b5a1dc592eb3 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java @@ -0,0 +1,320 @@ +/* + * 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.common; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.net.HostAndPort; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodList; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.utils.Utils; +import org.apache.commons.io.input.ReaderInputStream; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.io.InputStream; +import java.io.Reader; +import java.net.Socket; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +/** + * A Kubernetes client wrapper to assist with peon task managment. + * It provides a high level api to retreive jobs, launch jobs, delete jobs and various other + * tasks like getting task logs, listing all active tasks. + */ + +public class DruidKubernetesPeonClient implements KubernetesPeonClient +{ + + private static final EmittingLogger log = new EmittingLogger(DruidKubernetesPeonClient.class); + + private final KubernetesClientApi clientApi; + private final String namespace; + private final boolean debugJobs; + + public DruidKubernetesPeonClient(KubernetesClientApi clientApi, String namespace, boolean debugJobs) + { + this.clientApi = clientApi; + this.namespace = namespace; + this.debugJobs = debugJobs; + } + + @Override + public Optional jobExists(K8sTaskId taskId) + { + return clientApi.executeRequest( + client -> { + return Optional.fromNullable( + client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sTaskId()) + .get()); + } + ); + } + + @Override + public Pod launchJobAndWaitForStart(Job job, long howLong, TimeUnit timeUnit) + { + long start = System.currentTimeMillis(); + // launch job + return clientApi.executeRequest(client -> { + client.batch().v1().jobs().inNamespace(namespace).create(job); + K8sTaskId taskId = new K8sTaskId(job.getMetadata().getName()); + log.info("Successfully submitted job: %s ... waiting for job to launch", taskId); + // wait until the pod is running or complete or failed, any of those is fine + Pod mainPod = getMainJobPod(taskId); + Pod result = client.pods().inNamespace(namespace).withName(mainPod.getMetadata().getName()) + .waitUntilCondition(pod -> { + if (pod == null) { + return false; + } + return pod.getStatus() != null && pod.getStatus().getPodIP() != null; + }, howLong, timeUnit); + long duration = System.currentTimeMillis() - start; + log.info("Took task %s %d ms for pod to startup", taskId, duration); + return result; + }); + } + + @Override + public boolean waitForProcessToStart(TaskLocation location, long howLong, TimeUnit timeUnit) + { + CompletableFuture future = CompletableFuture.runAsync(() -> { + boolean scanning = true; + HostAndPort hostAndPort = location.toHostAndPort(); + while (scanning) { + try { + new Socket(hostAndPort.getHostText(), hostAndPort.getPort()).close(); + scanning = false; + } + catch (Exception ignore) { + } + } + }); + return Utils.waitUntilReady(future, howLong, timeUnit); + } + + @Override + public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) + { + return clientApi.executeRequest(client -> { + Job job = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sTaskId()) + .waitUntilCondition( + x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null, + howLong, + unit + ); + if (job.getStatus().getSucceeded() != null) { + return new JobResponse(job, PeonPhase.SUCCEEDED); + } + return new JobResponse(job, PeonPhase.FAILED); + }); + } + + @Override + public boolean cleanUpJob(K8sTaskId taskId) + { + if (!debugJobs) { + Boolean result = clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sTaskId()) + .delete()); + if (result) { + log.info("Cleaned up k8s task: %s", taskId); + } else { + log.info("Failed to cleanup task: %s", taskId); + } + return result; + } else { + log.info("Not cleaning up task %s due to flag: debugJobs=true", taskId); + return true; + } + + } + + @Override + public String getJobLogs(K8sTaskId taskId) + { + try { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sTaskId()) + .inContainer("main") + .getLog(true)); + } + catch (Exception e) { + return "No logs found: " + e; + } + } + + @Override + public Optional getPeonLogs(K8sTaskId taskId) + { + try { + return clientApi.executeRequest(client -> { + Reader reader = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sTaskId()) + .getLogReader(); + if (reader == null) { + return Optional.absent(); + } + return Optional.of(new ReaderInputStream(reader, StandardCharsets.UTF_8)); + }); + } + catch (Exception e) { + log.error("Error streaming logs from task: %s", taskId); + return Optional.absent(); + } + } + + @Override + public List listAllPeonJobs() + { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .list() + .getItems()); + } + + @Override + public List listPeonPods(Set phases) + { + return listPeonPods().stream() + .filter(x -> phases.contains(PeonPhase.getPhaseFor(x.getStatus().getPhase()))) + .collect(Collectors.toList()); + } + + @Override + public List listPeonPods() + { + PodList podList = clientApi.executeRequest(client -> client.pods().inNamespace(namespace)) + .withLabel(DruidK8sConstants.LABEL_KEY) + .list(); + return podList.getItems(); + } + + @Override + public int cleanCompletedJobsOlderThan(long howFarBack, TimeUnit timeUnit) + { + AtomicInteger numDeleted = new AtomicInteger(); + return clientApi.executeRequest(client -> { + List jobs = getJobsToCleanup(listAllPeonJobs(), howFarBack, timeUnit); + jobs.forEach(x -> { + if (client.batch().v1().jobs().inNamespace(namespace).withName(x.getMetadata().getName()).delete()) { + numDeleted.incrementAndGet(); + } + }); + return numDeleted.get(); + }); + } + + @Override + public Pod getMainJobPod(K8sTaskId taskId) + { + return clientApi.executeRequest(client -> getMainJobPod(client, taskId)); + } + + + @VisibleForTesting + List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit) + { + List toDelete = new ArrayList<>(); + long cutOff = System.currentTimeMillis() - timeUnit.toMillis(howFarBack); + candidates.forEach(x -> { + // jobs that are complete + if (x.getStatus().getActive() == null) { + Timestamp timestamp = Timestamp.valueOf(x.getStatus().getCompletionTime()); + if (timestamp.before(new Timestamp(cutOff))) { + toDelete.add(x); + } + } + }); + return toDelete; + } + + private Pod waitForPhase(KubernetesClient client, K8sTaskId taskId, long howLong, TimeUnit unit, PeonPhase... phase) + { + Pod mainPod = getMainJobPod(client, taskId); + client.pods().inNamespace(namespace).withName(mainPod.getMetadata().getName()) + .waitUntilCondition(pod -> { + if (pod == null) { + return false; + } + PeonPhase match = Arrays.stream(phase) + .filter(x -> x.equals(PeonPhase.getPhaseFor(pod.getStatus().getPhase()))) + .findAny().orElse(null); + return match != null; + }, howLong, unit); + return mainPod; + } + + Pod getMainJobPod(KubernetesClient client, K8sTaskId taskId) + { + String k8sTaskId = taskId.getK8sTaskId(); + try { + return RetryUtils.retry( + () -> { + PodList list = client.pods().inNamespace(namespace).withLabel("job-name", k8sTaskId).list(); + if (list.getItems().size() > 0) { + return list.getItems().get(0); + } + throw new KubernetesResourceNotFoundException( + "K8s pod with label: job-name=" + + k8sTaskId + + " not found"); + }, + DruidK8sConstants.IS_TRANSIENT, 5, RetryUtils.DEFAULT_MAX_TRIES + ); + } + catch (Exception e) { + throw new KubernetesResourceNotFoundException("K8s pod with label: job-name=" + k8sTaskId + " not found"); + } + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java new file mode 100644 index 000000000000..fcb810e307bb --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java @@ -0,0 +1,92 @@ +/* + * 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.common; + +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.joda.time.Period; +import org.joda.time.PeriodType; + + +public class JobResponse +{ + + private static final EmittingLogger LOGGER = new EmittingLogger(JobResponse.class); + + private final Job job; + private final PeonPhase phase; + private TaskLocation location; + + public JobResponse(Job job, PeonPhase phase) + { + this.job = job; + this.phase = phase; + } + + public Job getJob() + { + return job; + } + + public PeonPhase getPhase() + { + return phase; + } + + public TaskLocation getLocation() + { + return location; + } + + public void setLocation(TaskLocation location) + { + this.location = location; + } + + public Optional getJobDuration() + { + Optional duration = Optional.absent(); + try { + if (job.getStatus() != null) { + if (job.getStatus().getStartTime() != null) { + if (job.getStatus().getCompletionTime() != null) { + duration = Optional.of((long) new Period( + DateTimes.of(job.getStatus().getStartTime()), + DateTimes.of(job.getStatus().getCompletionTime()), + PeriodType.millis() + ).getMillis()); + } + } + } + } + catch (Exception e) { + LOGGER.error(e, "Error calculating duration for job: %s", job.getMetadata().getName()); + } + if (duration.isPresent()) { + LOGGER.info("Duration for Job: %s was %d seconds", job.getMetadata().getName(), duration.get()); + } else { + LOGGER.info("Unable to calcuate duration for Job: %s", job.getMetadata().getName()); + } + return duration; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java new file mode 100644 index 000000000000..89f0102d9432 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java @@ -0,0 +1,277 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerPort; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.EnvVarBuilder; +import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder; +import io.fabric8.kubernetes.api.model.ObjectFieldSelector; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.PodTemplateSpec; +import io.fabric8.kubernetes.api.model.Quantity; +import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; +import org.apache.commons.lang3.StringUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * This class transforms tasks to pods, and pods to tasks to assist with creating the job spec for a + * peon task. + * The two subclasses of this class are the SingleContainerTaskAdapter and the MultiContainerTaskAdapter + * This class runs on the overlord, to convert a task into a job, it will take its own podSpec (the current running overlord) + * keep volumees, secrets, env variables, config maps, etc..and add some additional information as well as provide a new + * command for running the task. + * The SingleContainerTaskAdapter only runs a task in a single container (no sidecars) + * The MultiContainerTaskAdapter runs with all of the sidecars the current running overlord runs with. Thus it needs + * to add some extra coordination to shutdown sidecar containers when the main pod exits. + */ + +public abstract class K8sTaskAdapter implements TaskAdapter +{ + + private static final EmittingLogger log = new EmittingLogger(K8sTaskAdapter.class); + + protected final KubernetesClientApi client; + protected final KubernetesTaskRunnerConfig config; + protected final ObjectMapper mapper; + + public K8sTaskAdapter( + KubernetesClientApi client, + KubernetesTaskRunnerConfig config, + ObjectMapper mapper + ) + { + this.client = client; + this.config = config; + this.mapper = mapper; + } + + @Override + public Job fromTask(Task task, PeonCommandContext context) throws IOException + { + String myPodName = System.getenv("HOSTNAME"); + Pod pod = client.executeRequest(client -> client.pods().inNamespace(config.namespace).withName(myPodName).get()); + return createJobFromPodSpec(pod.getSpec(), task, context); + } + + @Override + public Task toTask(Pod from) throws IOException + { + List envVars = from.getSpec().getContainers().get(0).getEnv(); + Optional taskJson = envVars.stream().filter(x -> "TASK_JSON".equals(x.getName())).findFirst(); + String contents = taskJson.map(envVar -> taskJson.get().getValue()).orElse(null); + if (contents == null) { + throw new IOException("No TASK_JSON environment variable found in pod: " + from.getMetadata().getName()); + } + return mapper.readValue(Base64Compression.decompressBase64(contents), Task.class); + } + + @VisibleForTesting + public abstract Job createJobFromPodSpec(PodSpec podSpec, Task task, PeonCommandContext context) throws IOException; + + protected Job buildJob( + K8sTaskId k8sTaskId, + Map labels, + Map annotations, PodTemplateSpec podTemplate + ) + { + return new JobBuilder() + .withNewMetadata() + .withName(k8sTaskId.getK8sTaskId()) + .addToLabels(labels) + .addToAnnotations(annotations) + .endMetadata() + .withNewSpec() + .withTemplate(podTemplate) + .withActiveDeadlineSeconds(config.maxTaskDuration.toStandardDuration().getStandardSeconds()) + .withBackoffLimit(0) + .withTtlSecondsAfterFinished((int) config.taskCleanupDelay.toStandardDuration().getStandardSeconds()) + .endSpec() + .build(); + } + + @VisibleForTesting + static Optional getJavaOptValueBytes(String qualifier, List commands) + { + Long result = null; + Optional lastHeapValue = commands.stream().filter(x -> x.startsWith(qualifier)).reduce((x, y) -> y); + if (lastHeapValue.isPresent()) { + result = HumanReadableBytes.parse(StringUtils.removeStart(lastHeapValue.get(), qualifier)); + } + return Optional.ofNullable(result); + } + + @VisibleForTesting + static long getContainerMemory(PeonCommandContext context) + { + List javaOpts = context.getJavaOpts(); + Optional optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts); + long heapSize = HumanReadableBytes.parse("1g"); + if (optionalXmx.isPresent()) { + heapSize = optionalXmx.get(); + } + Optional optionalDbb = getJavaOptValueBytes("-XX:MaxDirectMemorySize=", javaOpts); + long dbbSize = heapSize; + if (optionalDbb.isPresent()) { + dbbSize = optionalDbb.get(); + } + return (long) ((dbbSize + heapSize) * 1.2); + + } + + protected void setupPorts(Container mainContainer) + { + mainContainer.getPorts().clear(); + ContainerPort tcpPort = new ContainerPort(); + tcpPort.setContainerPort(DruidK8sConstants.PORT); + tcpPort.setName("druid-port"); + tcpPort.setProtocol("TCP"); + ContainerPort httpsPort = new ContainerPort(); + httpsPort.setContainerPort(DruidK8sConstants.TLS_PORT); + httpsPort.setName("druid-tls-port"); + httpsPort.setProtocol("TCP"); + mainContainer.setPorts(Lists.newArrayList(httpsPort, tcpPort)); + } + + protected void addEnvironmentVariables(Container mainContainer, PeonCommandContext context, String taskContents) + { + mainContainer.getEnv().addAll(Lists.newArrayList( + new EnvVarBuilder() + .withName(DruidK8sConstants.TASK_DIR_ENV) + .withValue(context.getTaskDir().getAbsolutePath()) + .build(), + new EnvVarBuilder() + .withName(DruidK8sConstants.TASK_JSON_ENV) + .withValue(taskContents) + .build(), + new EnvVarBuilder() + .withName(DruidK8sConstants.JAVA_OPTS) + .withValue(Joiner.on(" ").join(context.getJavaOpts())) + .build(), + new EnvVarBuilder() + .withName(DruidK8sConstants.DRUID_HOST_ENV) + .withValueFrom(new EnvVarSourceBuilder().withFieldRef(new ObjectFieldSelector( + null, + "status.podIP" + )).build()).build(), + new EnvVarBuilder() + .withName(DruidK8sConstants.DRUID_HOSTNAME_ENV) + .withValueFrom(new EnvVarSourceBuilder().withFieldRef(new ObjectFieldSelector( + null, + "metadata.name" + )).build()).build() + )); + } + + protected Container setupMainContainer( + PodSpec podSpec, + PeonCommandContext context, + long containerSize, + String taskContents + ) + { + // prepend the startup task.json extraction command + List mainCommand = Lists.newArrayList("sh", "-c"); + // update the command + List containers = podSpec.getContainers(); + Container mainContainer = Iterables.getFirst(containers, null); + if (mainContainer == null) { + throw new IllegalArgumentException("Must have at least one container"); + } + + // remove probes + mainContainer.setReadinessProbe(null); + mainContainer.setLivenessProbe(null); + + setupPorts(mainContainer); + addEnvironmentVariables(mainContainer, context, taskContents); + + mainContainer.setCommand(mainCommand); + mainContainer.setArgs(Collections.singletonList(Joiner.on(" ").join(context.getComamnd()))); + + mainContainer.setName("main"); + ImmutableMap resources = ImmutableMap.of( + "cpu", + new Quantity("1000", "m"), + "memory", + new Quantity(String.valueOf(containerSize)) + ); + mainContainer.setResources(new ResourceRequirementsBuilder().withRequests(resources).withLimits(resources).build()); + return mainContainer; + } + + protected Map addJobSpecificAnnotations(PeonCommandContext context, K8sTaskId k8sTaskId) + { + Map annotations = config.annotations; + annotations.put(DruidK8sConstants.TASK_ID, k8sTaskId.getOriginalTaskId()); + annotations.put(DruidK8sConstants.TLS_ENABLED, String.valueOf(context.isEnableTls())); + return annotations; + } + + protected Map addJobSpecificLabels() + { + Map labels = config.labels; + labels.put(DruidK8sConstants.LABEL_KEY, "true"); + return labels; + } + + protected PodTemplateSpec createTemplateFromSpec( + K8sTaskId k8sTaskId, + PodSpec podSpec, + Map annotations, + Map labels + ) + { + // clean up the podSpec + podSpec.setNodeName(null); + podSpec.setRestartPolicy("Never"); + podSpec.setHostname(k8sTaskId.getK8sTaskId()); + podSpec.setTerminationGracePeriodSeconds(config.graceTerminationPeriodSeconds); + + PodTemplateSpec podTemplate = new PodTemplateSpec(); + ObjectMeta objectMeta = new ObjectMeta(); + objectMeta.setAnnotations(annotations); + objectMeta.setLabels(labels); + podTemplate.setMetadata(objectMeta); + podTemplate.setSpec(podSpec); + return podTemplate; + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskId.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskId.java new file mode 100644 index 000000000000..dec661333577 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskId.java @@ -0,0 +1,82 @@ +/* + * 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.common; + +import org.apache.commons.lang3.RegExUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.druid.indexing.common.task.Task; + +import java.util.Locale; +import java.util.Objects; + +public class K8sTaskId +{ + + private final String k8sTaskId; + private final String originalTaskId; + + public K8sTaskId(Task task) + { + this(task.getId()); + } + + public K8sTaskId(String taskId) + { + this.originalTaskId = taskId; + // replace all the ": - . _" to "", try to reduce the length of pod name and meet pod naming specifications 64 characters. + this.k8sTaskId = StringUtils.left(RegExUtils.replaceAll(taskId, "[^a-zA-Z0-9\\\\s]", "") + .toLowerCase(Locale.ENGLISH), 63); + } + + public String getK8sTaskId() + { + return k8sTaskId; + } + + public String getOriginalTaskId() + { + return originalTaskId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + K8sTaskId k8sTaskId1 = (K8sTaskId) o; + return k8sTaskId.equals(k8sTaskId1.k8sTaskId) && originalTaskId.equals(k8sTaskId1.originalTaskId); + } + + @Override + public int hashCode() + { + return Objects.hash(k8sTaskId, originalTaskId); + } + + @Override + public String toString() + { + return "[ " + originalTaskId + ", " + k8sTaskId + "]"; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java new file mode 100644 index 000000000000..5822b850174b --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +public interface KubernetesClientApi +{ + T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException; +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesExecutor.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesExecutor.java new file mode 100644 index 000000000000..268328ff6f8c --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesExecutor.java @@ -0,0 +1,28 @@ +/* + * 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.common; + +import io.fabric8.kubernetes.client.KubernetesClient; + +@FunctionalInterface +public interface KubernetesExecutor +{ + T executeRequest(KubernetesClient client) throws 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 new file mode 100644 index 000000000000..d229213292ea --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -0,0 +1,60 @@ +/* + * 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.common; + +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.druid.indexer.TaskLocation; + +import java.io.InputStream; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +public interface KubernetesPeonClient +{ + + Optional jobExists(K8sTaskId taskId); + + Pod launchJobAndWaitForStart(Job job, long howLong, TimeUnit timeUnit); + + boolean waitForProcessToStart(TaskLocation location, long howLong, TimeUnit timeUnit); + + JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit timeUnit); + + boolean cleanUpJob(K8sTaskId taskId); + + String getJobLogs(K8sTaskId taskId); + + Optional getPeonLogs(K8sTaskId taskId); + + List listAllPeonJobs(); + + List listPeonPods(Set phases); + + List listPeonPods(); + + int cleanCompletedJobsOlderThan(long howFarBack, TimeUnit timeUnit); + + Pod getMainJobPod(K8sTaskId taskId); + + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceNotFoundException.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceNotFoundException.java new file mode 100644 index 000000000000..af3f4e7cceb7 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceNotFoundException.java @@ -0,0 +1,28 @@ +/* + * 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.common; + +public class KubernetesResourceNotFoundException extends RuntimeException +{ + public KubernetesResourceNotFoundException(String message) + { + super(message); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java new file mode 100644 index 000000000000..b3f4fb104da6 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java @@ -0,0 +1,151 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.PodTemplateSpec; +import io.fabric8.kubernetes.api.model.Volume; +import io.fabric8.kubernetes.api.model.VolumeBuilder; +import io.fabric8.kubernetes.api.model.VolumeMount; +import io.fabric8.kubernetes.api.model.VolumeMountBuilder; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MultiContainerTaskAdapter extends K8sTaskAdapter +{ + public MultiContainerTaskAdapter( + KubernetesClientApi client, + KubernetesTaskRunnerConfig config, + ObjectMapper mapper + ) + { + super(client, config, mapper); + } + + @Override + public Job createJobFromPodSpec(PodSpec podSpec, Task task, PeonCommandContext context) throws IOException + { + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + // get the container size from java_opts array + long containerSize = getContainerMemory(context); + + // compress the task.json to set as an env variables + String taskContents = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + + setupMainContainer(podSpec, context, containerSize, taskContents); + + // add any optional annotations or labels. + Map annotations = addJobSpecificAnnotations(context, k8sTaskId); + Map labels = addJobSpecificLabels(); + + PodTemplateSpec podTemplate = createTemplateFromSpec(k8sTaskId, podSpec, annotations, labels); + + // add sidecar termination support + addSideCarTerminationSupport(podTemplate); + + // and the init container + podTemplate.getSpec().getInitContainers().add(getInitContainer()); + + // create the job + return buildJob(k8sTaskId, labels, annotations, podTemplate); + } + + @VisibleForTesting + private Container getInitContainer() + { + return new ContainerBuilder() + .withName("kubexit") + .withImage(config.kubexitImage) + .withCommand("cp", "/bin/kubexit", "/kubexit/kubexit") + .withVolumeMounts(new VolumeMountBuilder().withMountPath("/kubexit").withName("kubexit").build()) + .build(); + } + + static void reJiggerArgsAndCommand(Container container, boolean primary) + { + List originalCommand = container.getCommand(); + List originalArgs = container.getArgs(); + originalCommand.addAll(originalArgs); + String newArgs; + if (primary) { + // for primary the command is /bin/sh -c, don't need this again, only grab args + newArgs = Joiner.on(" ").join(originalArgs); + } else { + newArgs = Joiner.on(" ").join(originalCommand); + } + container.setCommand(Lists.newArrayList("/bin/sh", "-c")); + String toExecute = "/kubexit/kubexit /bin/sh -c " + "\"" + StringEscapeUtils.escapeJava(newArgs) + "\""; + // we don't care about exit code of sidecar containers + if (!primary) { + toExecute += " || true"; + } + container.setArgs(Collections.singletonList(toExecute)); + } + + static void addSideCarTerminationSupport(PodTemplateSpec spec) + { + Volume graveyard = new VolumeBuilder().withName("graveyard") + .withNewEmptyDir() + .withMedium("Memory") + .endEmptyDir() + .build(); + Volume kubeExit = new VolumeBuilder().withName("kubexit") + .withNewEmptyDir() + .endEmptyDir() + .build(); + spec.getSpec().getVolumes().add(graveyard); + spec.getSpec().getVolumes().add(kubeExit); + + VolumeMount gMount = new VolumeMountBuilder().withMountPath("/graveyard").withName("graveyard").build(); + VolumeMount kMount = new VolumeMountBuilder().withMountPath("/kubexit").withName("kubexit").build(); + + + // get the main container + List containers = spec.getSpec().getContainers(); + for (int i = 0; i < containers.size(); i++) { + Container container = containers.get(i); + container.getEnv().add(new EnvVar("KUBEXIT_NAME", container.getName(), null)); + container.getEnv().add(new EnvVar("KUBEXIT_GRAVEYARD", "/graveyard", null)); + container.getVolumeMounts().add(gMount); + container.getVolumeMounts().add(kMount); + if (i > 0) { + container.getEnv().add(new EnvVar("KUBEXIT_DEATH_DEPS", containers.get(0).getName(), null)); + reJiggerArgsAndCommand(container, false); + } else { + reJiggerArgsAndCommand(container, true); + } + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonCommandContext.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonCommandContext.java new file mode 100644 index 000000000000..8984a73bd1ba --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonCommandContext.java @@ -0,0 +1,73 @@ +/* + * 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.common; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class PeonCommandContext +{ + + private final List comamnd; + private final List javaOpts; + private final File taskDir; + private final boolean enableTls; + + public PeonCommandContext(List comamnd, List javaOpts, File taskDir) + { + this(comamnd, javaOpts, taskDir, false); + } + + public PeonCommandContext(List comamnd, List javaOpts, File taskDir, boolean enableTls) + { + this.comamnd = comamnd; + this.javaOpts = javaOpts; + this.taskDir = taskDir; + this.enableTls = enableTls; + } + + public List getComamnd() + { + return comamnd; + } + + public List getJavaOpts() + { + // we don't know if they put everything in as one string, or split. + List result = new ArrayList<>(); + for (String javaOpt : javaOpts) { + String[] value = javaOpt.split("\\s+"); + result.addAll(Arrays.asList(value)); + } + return result; + } + + public File getTaskDir() + { + return taskDir; + } + + public boolean isEnableTls() + { + return enableTls; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java new file mode 100644 index 000000000000..5cf0ac71e8c8 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PeonPhase.java @@ -0,0 +1,67 @@ +/* + * 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.common; + +import io.fabric8.kubernetes.api.model.Pod; + +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public enum PeonPhase +{ + PENDING("Pending"), + SUCCEEDED("Succeeded"), + FAILED("Failed"), + UNKNOWN("Unknown"), + RUNNING("Running"); + + private static final Map PHASE_MAP = Arrays.stream(PeonPhase.values()) + .collect(Collectors.toMap( + PeonPhase::getPhase, + Function.identity() + )); + private final String phase; + + PeonPhase(String phase) + { + this.phase = phase; + } + + public String getPhase() + { + return phase; + } + + public static PeonPhase getPhaseFor(String k8sPhase) + { + return PHASE_MAP.getOrDefault(k8sPhase, UNKNOWN); + } + + public static PeonPhase getPhaseFor(Pod pod) + { + if (pod == null) { + return UNKNOWN; + } + return PHASE_MAP.get(pod.getStatus().getPhase()); + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java new file mode 100644 index 000000000000..a86fed2fc2ef --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java @@ -0,0 +1,68 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +public class SingleContainerTaskAdapter extends K8sTaskAdapter +{ + public SingleContainerTaskAdapter( + KubernetesClientApi client, + KubernetesTaskRunnerConfig config, + ObjectMapper mapper + ) + { + super(client, config, mapper); + } + + @Override + public Job createJobFromPodSpec(PodSpec podSpec, Task task, PeonCommandContext context) throws IOException + { + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + // get the container size from java_opts array + long containerSize = getContainerMemory(context); + + // compress the task.json to set as an env variables + String taskContents = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + + Container mainContainer = setupMainContainer(podSpec, context, containerSize, taskContents); + + // add any optional annotations or labels. + Map annotations = addJobSpecificAnnotations(context, k8sTaskId); + Map labels = addJobSpecificLabels(); + + // remove all sidecars + podSpec.setContainers(Collections.singletonList(mainContainer)); + + // create the job + return buildJob(k8sTaskId, labels, annotations, createTemplateFromSpec(k8sTaskId, podSpec, annotations, labels)); + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java new file mode 100644 index 000000000000..5033f4af12eb --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java @@ -0,0 +1,33 @@ +/* + * 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.common; + +import org.apache.druid.indexing.common.task.Task; + +import java.io.IOException; + +public interface TaskAdapter +{ + + V fromTask(Task task, PeonCommandContext context) throws IOException; + + Task toTask(K from) throws IOException; + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..fd4d44540835 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# 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. + +org.apache.druid.k8s.overlord.K8sOverlordModule diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java new file mode 100644 index 000000000000..06c69875ef24 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -0,0 +1,439 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodStatus; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobStatus; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.common.DruidKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.JobResponse; +import org.apache.druid.k8s.overlord.common.K8sTaskAdapter; +import org.apache.druid.k8s.overlord.common.K8sTaskId; +import org.apache.druid.k8s.overlord.common.PeonCommandContext; +import org.apache.druid.k8s.overlord.common.PeonPhase; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogPusher; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class KubernetesTaskRunnerTest +{ + + private TaskQueueConfig taskQueueConfig; + private StartupLoggingConfig startupLoggingConfig; + private ObjectMapper jsonMapper; + private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private TaskConfig taskConfig; + private TaskLogPusher taskLogPusher; + private DruidNode node; + + public KubernetesTaskRunnerTest() + { + TestUtils utils = new TestUtils(); + jsonMapper = utils.getTestObjectMapper(); + for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + } + jsonMapper.registerSubtypes( + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(IndexTask.IndexTuningConfig.class, "index") + ); + } + + @BeforeEach + void setUp() + { + taskConfig = new TaskConfig( + "src/test/resources", + "src/test/resources", + null, + null, + null, + false, + null, + null, + null, + false, + false, + null, + null, + false + ); + kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig(); + kubernetesTaskRunnerConfig.namespace = "test"; + kubernetesTaskRunnerConfig.javaOptsArray = Collections.singletonList("-Xmx2g"); + taskQueueConfig = new TaskQueueConfig(1, Period.millis(1), Period.millis(1), Period.millis(1)); + startupLoggingConfig = new StartupLoggingConfig(); + taskLogPusher = mock(TaskLogPusher.class); + node = mock(DruidNode.class); + when(node.isEnableTlsPort()).thenReturn(false); + } + + @Test + void testAlreadyRunningJobInK8s() throws Exception + { + Task task = makeTask(); + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("jobName"); + + Job job = mock(Job.class); + JobStatus status = mock(JobStatus.class); + when(job.getMetadata()).thenReturn(metadata); + when(status.getActive()).thenReturn(1).thenReturn(null); + when(job.getStatus()).thenReturn(status); + + Pod peonPod = mock(Pod.class); + when(peonPod.getMetadata()).thenReturn(metadata); + PodStatus podStatus = mock(PodStatus.class); + when(podStatus.getPodIP()).thenReturn("SomeIP"); + when(peonPod.getStatus()).thenReturn(podStatus); + + K8sTaskAdapter adapter = mock(K8sTaskAdapter.class); + when(adapter.fromTask(eq(task), any())).thenReturn(job); + + DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class); + + when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job)); + when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod); + when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true); + + KubernetesTaskRunner taskRunner = new KubernetesTaskRunner( + taskConfig, + startupLoggingConfig, + adapter, + kubernetesTaskRunnerConfig, + taskQueueConfig, + taskLogPusher, + peonClient, + node + ); + KubernetesTaskRunner spyRunner = spy(taskRunner); + + ListenableFuture future = spyRunner.run(task); + future.get(); + // we should never launch the job here, one exists + verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class)); + verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId)); + TaskLocation expectedTaskLocation = TaskLocation.create( + peonPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT, + node.isEnableTlsPort() + ); + verify(spyRunner, times(1)).updateLocation(eq(task), eq(expectedTaskLocation)); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.running(task.getId()))); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId(), expectedTaskLocation))); + } + + @Test + void testJobNeedsToLaunchInK8s() throws Exception + { + Task task = makeTask(); + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + Job job = mock(Job.class); + ObjectMeta jobMetadata = mock(ObjectMeta.class); + when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId()); + JobStatus status = mock(JobStatus.class); + when(status.getActive()).thenReturn(1).thenReturn(null); + when(job.getStatus()).thenReturn(status); + when(job.getMetadata()).thenReturn(jobMetadata); + + Pod peonPod = mock(Pod.class); + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("peonPodName"); + when(peonPod.getMetadata()).thenReturn(metadata); + PodStatus podStatus = mock(PodStatus.class); + when(podStatus.getPodIP()).thenReturn("SomeIP"); + when(peonPod.getStatus()).thenReturn(podStatus); + + K8sTaskAdapter adapter = mock(K8sTaskAdapter.class); + when(adapter.fromTask(eq(task), isA(PeonCommandContext.class))).thenReturn(job); + + DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class); + + when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null)); + when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod); + when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod); + when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true); + + KubernetesTaskRunner taskRunner = new KubernetesTaskRunner( + taskConfig, + startupLoggingConfig, + adapter, + kubernetesTaskRunnerConfig, + taskQueueConfig, + taskLogPusher, + peonClient, + node + ); + KubernetesTaskRunner spyRunner = spy(taskRunner); + + + ListenableFuture future = spyRunner.run(task); + future.get(); + // we should never launch the job here, one exists + verify(peonClient, times(1)).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class)); + verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId)); + TaskLocation expectedTaskLocation = TaskLocation.create( + peonPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT, + node.isEnableTlsPort() + ); + verify(spyRunner, times(1)).updateLocation(eq(task), eq(expectedTaskLocation)); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.running(task.getId()))); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId(), expectedTaskLocation))); + } + + @Test + void testTheK8sRestartState() throws Exception + { + // we have a shutdown, now we start-up the overlord, it should catch and deal with all the peon k8s tasks in-flight + Task task = makeTask(); + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + Job job = mock(Job.class); + ObjectMeta jobMetadata = mock(ObjectMeta.class); + when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId()); + JobStatus status = mock(JobStatus.class); + when(status.getActive()).thenReturn(1).thenReturn(null); + when(job.getStatus()).thenReturn(status); + when(job.getMetadata()).thenReturn(jobMetadata); + + Pod peonPod = mock(Pod.class); + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("peonPodName"); + when(metadata.getCreationTimestamp()).thenReturn(DateTimes.nowUtc().toString()); + when(peonPod.getMetadata()).thenReturn(metadata); + PodStatus podStatus = mock(PodStatus.class); + when(podStatus.getPodIP()).thenReturn("SomeIP"); + when(peonPod.getStatus()).thenReturn(podStatus); + + K8sTaskAdapter adapter = mock(K8sTaskAdapter.class); + when(adapter.fromTask(eq(task), isA(PeonCommandContext.class))).thenReturn(job); + when(adapter.toTask(eq(peonPod))).thenReturn(task); + + DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class); + + when(peonClient.listPeonPods()).thenReturn(Lists.newArrayList(peonPod)); + when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job)); + when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod); + when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod); + when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true); + + KubernetesTaskRunner taskRunner = new KubernetesTaskRunner( + taskConfig, + startupLoggingConfig, + adapter, + kubernetesTaskRunnerConfig, + taskQueueConfig, + taskLogPusher, + peonClient, + node + ); + KubernetesTaskRunner spyRunner = spy(taskRunner); + Collection workItems = spyRunner.getKnownTasks(); + Assertions.assertEquals(1, workItems.size()); + TaskRunnerWorkItem item = Iterables.getOnlyElement(workItems); + item.getResult().get(); + + // we should never launch the job here, one exists + verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class)); + verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId)); + TaskLocation expectedTaskLocation = TaskLocation.create( + peonPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT, + node.isEnableTlsPort() + ); + verify(spyRunner, times(1)).updateLocation(eq(task), eq(expectedTaskLocation)); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.running(task.getId()))); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId(), expectedTaskLocation))); + verify(spyRunner, times(1)).run(eq(task)); + } + + @Test + void testTheK8sRestartStateAndHandleJobsThatAlreadyCompletedWhileDown() throws Exception + { + // we have a shutdown, now we start-up the overlord, it should monitor k8s jobs that finished. + Task task = makeTask(); + K8sTaskId k8sTaskId = new K8sTaskId(task.getId()); + + Job job = mock(Job.class); + ObjectMeta jobMetadata = mock(ObjectMeta.class); + when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId()); + JobStatus status = mock(JobStatus.class); + when(status.getActive()).thenReturn(null); + when(job.getStatus()).thenReturn(status); + when(job.getStatus().getSucceeded()).thenReturn(1); + when(job.getMetadata()).thenReturn(jobMetadata); + + Pod peonPod = mock(Pod.class); + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("peonPodName"); + when(metadata.getCreationTimestamp()).thenReturn(DateTimes.nowUtc().toString()); + when(peonPod.getMetadata()).thenReturn(metadata); + PodStatus podStatus = mock(PodStatus.class); + when(podStatus.getPodIP()).thenReturn("SomeIP"); + when(peonPod.getStatus()).thenReturn(podStatus); + + K8sTaskAdapter adapter = mock(K8sTaskAdapter.class); + when(adapter.fromTask(eq(task), isA(PeonCommandContext.class))).thenReturn(job); + when(adapter.toTask(eq(peonPod))).thenReturn(task); + + DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class); + + when(peonClient.listPeonPods()).thenReturn(Lists.newArrayList(peonPod)); + when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job)); + when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod); + when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod); + when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(job, PeonPhase.SUCCEEDED)); + when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true); + + KubernetesTaskRunner taskRunner = new KubernetesTaskRunner( + taskConfig, + startupLoggingConfig, + adapter, + kubernetesTaskRunnerConfig, + taskQueueConfig, + taskLogPusher, + peonClient, + node + ); + KubernetesTaskRunner spyRunner = spy(taskRunner); + Collection workItems = spyRunner.getKnownTasks(); + Assertions.assertEquals(1, workItems.size()); + TaskRunnerWorkItem item = Iterables.getOnlyElement(workItems); + item.getResult().get(); + + // we should never launch the job here, one exists + verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class)); + verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId)); + TaskLocation expectedTaskLocation = TaskLocation.create( + peonPod.getStatus().getPodIP(), + DruidK8sConstants.PORT, + DruidK8sConstants.TLS_PORT + ); + // don't need to update the location, the one in the db was correct when it launched, + verify(spyRunner, never()).updateLocation(eq(task), eq(expectedTaskLocation)); + // the state is still running, as it was before the overlord went down. + verify(spyRunner, never()).updateStatus(eq(task), eq(TaskStatus.running(task.getId()))); + verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId(), expectedTaskLocation))); + verify(spyRunner, times(1)).run(eq(task)); + } + + + private Task makeTask() + { + return new TestableNoopTask( + "k8sTaskId", + null, + null, + 0, + 0, + null, + null, + ImmutableMap.of("druid.indexer.runner.javaOpts", "abc", + "druid.indexer.fork.property.druid.processing.buffer.sizeBytes", "2048", + "druid.peon.pod.cpu", "1", + "druid.peon.pod.memory", "2G" + ) + ); + } + + private static class TestableNoopTask extends NoopTask + { + TestableNoopTask( + @JsonProperty("id") String id, + @JsonProperty("groupId") String groupId, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("runTime") long runTime, + @JsonProperty("isReadyTime") long isReadyTime, + @JsonProperty("isReadyResult") String isReadyResult, + @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("context") Map context + ) + { + super(id, groupId, dataSource, runTime, isReadyTime, isReadyResult, firehoseFactory, context); + } + + @Override + public String getNodeType() + { + return "ForkNodeType"; + } + + @Override + public boolean supportsQueries() + { + return true; + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/Base64CompressionTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/Base64CompressionTest.java new file mode 100644 index 000000000000..99dda7b2185c --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/Base64CompressionTest.java @@ -0,0 +1,48 @@ +/* + * 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.common; + +import org.junit.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Base64; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class Base64CompressionTest +{ + + @Test + public void testBase64Compression() throws IOException + { + String lotsOfRepeatingCharacters = "mmmmmmmmmmmmmmmmmmmmkaaaaayyyyyyyyyyyy"; + String compressed = Base64Compression.compressBase64(lotsOfRepeatingCharacters); + String uncompressed = Base64.getEncoder() + .encodeToString(lotsOfRepeatingCharacters.getBytes(StandardCharsets.UTF_8)); + assertTrue(compressed.length() < uncompressed.length()); + + // now decompres this + String result = Base64Compression.decompressBase64(compressed); + assertEquals(lotsOfRepeatingCharacters, result); + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java new file mode 100644 index 000000000000..4883316dbb93 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java @@ -0,0 +1,166 @@ +/* + * 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.common; + +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.PodTemplateSpec; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; +import io.fabric8.kubernetes.api.model.batch.v1.JobStatus; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClientTimeoutException; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import org.apache.druid.common.utils.SocketUtil; +import org.apache.druid.indexer.TaskLocation; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.InputStream; +import java.net.ServerSocket; +import java.sql.Timestamp; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +@EnableKubernetesMockClient(crud = true) +public class DruidKubernetesPeonClientTest +{ + + KubernetesClient client; + + @Test + void testWaitingForAPodToGetReadyThatDoesntExist() + { + DruidKubernetesPeonClient client = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test", + false + ); + Assertions.assertThrows(KubernetesClientTimeoutException.class, () -> { + client.waitForJobCompletion(new K8sTaskId("some-task"), 1, TimeUnit.SECONDS); + }); + } + + @Test + void testTheFlow() throws Exception + { + DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test", + false + ); + List currentJobs = peonClient.listAllPeonJobs(); + assertEquals(0, currentJobs.size()); + Job job = new JobBuilder() + .withNewMetadata() + .withName("job_name") + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewSpec() + .withTemplate(new PodTemplateSpec(new ObjectMeta(), K8sTestUtils.getDummyPodSpec())) + .endSpec().build(); + client.batch().v1().jobs().inNamespace("test").create(job); + currentJobs = peonClient.listAllPeonJobs(); + assertEquals(1, currentJobs.size()); + } + + @Test + void testCleanup() throws KubernetesResourceNotFoundException + { + DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test", + false + ); + Job active = mockJob(true, new Timestamp(System.currentTimeMillis())); + long tenMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(10); + Job dontKillYet = mockJob(false, new Timestamp(tenMinutesAgo)); + long oneHourAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(60); + Job killThisOne = mockJob(false, new Timestamp(oneHourAgo)); + + List jobs = Lists.newArrayList(active, dontKillYet, killThisOne); + List toDelete = peonClient.getJobsToCleanup(jobs, 30, TimeUnit.MINUTES); + assertEquals(1, toDelete.size()); // should only cleanup one job + assertEquals(killThisOne, Iterables.getOnlyElement(toDelete)); // should only cleanup one job + } + + @Test + void testWaitingForSocketToBeAvailable() throws Exception + { + DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient( + new TestKubernetesClient(this.client), "test", + false + ); + int openPort = SocketUtil.findOpenPortFrom(1); + + CompletableFuture future = CompletableFuture.supplyAsync(new Supplier() + { + @Override + public Boolean get() + { + return peonClient.waitForProcessToStart( + new TaskLocation("localhost", openPort, -1), + 10, + TimeUnit.SECONDS + ); + } + }); + + ServerSocket socket = new ServerSocket(openPort); + socket.accept(); + + Boolean result = future.get(10, TimeUnit.SECONDS); + assertTrue(result); + socket.close(); + } + + @Test + void watchingALogThatDoesntExist() + { + DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient( + new TestKubernetesClient(this.client), "test", + false + ); + Optional stream = peonClient.getPeonLogs(new K8sTaskId("foo")); + assertFalse(stream.isPresent()); + + String logs = peonClient.getJobLogs(new K8sTaskId("foo")); + assertTrue(logs.startsWith("No logs found")); + } + + private Job mockJob(boolean active, Timestamp timestamp) + { + Job job = mock(Job.class); + JobStatus status = mock(JobStatus.class); + if (active) { + when(status.getActive()).thenReturn(1); + } else { + when(status.getActive()).thenReturn(null); + when(status.getCompletionTime()).thenReturn(timestamp.toString()); + } + when(job.getStatus()).thenReturn(status); + return job; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java new file mode 100644 index 000000000000..72acaf42b1b9 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java @@ -0,0 +1,152 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; + +// must have a kind / minikube cluster installed and the image pushed to your repository +public class DruidPeonClientIntegrationTest +{ + private final KubernetesClientApi k8sClient; + private final DruidKubernetesPeonClient peonClient; + private final ObjectMapper jsonMapper; + + public DruidPeonClientIntegrationTest() + { + TestUtils utils = new TestUtils(); + jsonMapper = utils.getTestObjectMapper(); + for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + } + jsonMapper.registerSubtypes( + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(IndexTask.IndexTuningConfig.class, "index") + ); + k8sClient = new DruidKubernetesClient(); + peonClient = new DruidKubernetesPeonClient(k8sClient, "default", false); + } + + @Disabled + @Test + public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception + { + PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); + + Task task = K8sTestUtils.getTask(); + KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig(); + config.namespace = "default"; + K8sTaskAdapter adapter = new SingleContainerTaskAdapter(k8sClient, config, jsonMapper); + String taskBasePath = "/home/taskDir"; + PeonCommandContext context = new PeonCommandContext(Collections.singletonList( + "sleep 10; for i in `seq 1 1000`; do echo $i; done; exit 0" + ), new ArrayList<>(), new File(taskBasePath)); + + Job job = adapter.createJobFromPodSpec(podSpec, task, context); + + // launch the job and wait to start... + peonClient.launchJobAndWaitForStart(job, 1, TimeUnit.MINUTES); + + // there should be one job that is a k8s peon job that exists + List jobs = peonClient.listAllPeonJobs(); + assertEquals(1, jobs.size()); + + K8sTaskId taskId = new K8sTaskId(task.getId()); + InputStream peonLogs = peonClient.getPeonLogs(taskId).get(); + List expectedLogs = IntStream.range(1, 1001).boxed().collect(Collectors.toList()); + List actualLogs = new ArrayList<>(); + Thread thread = new Thread(() -> { + try { + actualLogs.addAll(IOUtils.readLines(peonLogs, "UTF-8") + .stream() + .map(Integer::parseInt) + .collect(Collectors.toList())); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + thread.start(); + + // assert that the env variable is corret + Task taskFromEnvVar = adapter.toTask(peonClient.getMainJobPod(new K8sTaskId(task.getId()))); + assertEquals(task, taskFromEnvVar); + + // now copy the task.json file from the pod and make sure its the same as our task.json we expected + Path downloadPath = Paths.get(tempDir.toAbsolutePath().toString(), "task.json"); + Pod mainJobPod = peonClient.getMainJobPod(taskId); + k8sClient.executeRequest(client -> { + client.pods() + .inNamespace("default") + .withName(mainJobPod.getMetadata().getName()) + .file(Paths.get(taskBasePath, "task.json").toString()) + .copy(downloadPath); + return null; + }); + + String taskJsonFromPod = FileUtils.readFileToString(new File(downloadPath.toString()), StandardCharsets.UTF_8); + Task taskFromPod = jsonMapper.readValue(taskJsonFromPod, Task.class); + assertEquals(task, taskFromPod); + + + JobResponse jobStatusResult = peonClient.waitForJobCompletion(taskId, 2, TimeUnit.MINUTES); + thread.join(); + assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase()); + // as long as there were no exceptions we are good! + assertEquals(expectedLogs, actualLogs); + // cleanup my job + assertTrue(peonClient.cleanUpJob(taskId)); + + // we cleaned up the job, none should exist + List existingJobs = peonClient.listAllPeonJobs(); + assertEquals(0, existingJobs.size()); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java new file mode 100644 index 000000000000..567bef4779f8 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java @@ -0,0 +1,65 @@ +/* + * 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.common; + +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobStatus; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class JobResponseTest +{ + + @Test + void testCompletionTime() + { + Job job = mock(Job.class); + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("job"); + when(job.getMetadata()).thenReturn(metadata); + JobStatus jobStatus = mock(JobStatus.class); + when(jobStatus.getStartTime()).thenReturn("2022-09-19T23:31:50Z"); + when(jobStatus.getCompletionTime()).thenReturn("2022-09-19T23:32:48Z"); + when(job.getStatus()).thenReturn(jobStatus); + JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED); + Optional duration = response.getJobDuration(); + Assertions.assertEquals(Long.valueOf(58000L), duration.get()); + } + + @Test + void testNoDuration() + { + Job job = mock(Job.class); + ObjectMeta metadata = mock(ObjectMeta.class); + when(metadata.getName()).thenReturn("job"); + when(job.getMetadata()).thenReturn(metadata); + JobStatus jobStatus = mock(JobStatus.class); + when(jobStatus.getStartTime()).thenReturn("2022-09-19T23:31:50Z"); + when(job.getStatus()).thenReturn(jobStatus); + JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED); + Optional duration = response.getJobDuration(); + Assertions.assertFalse(duration.isPresent()); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java new file mode 100644 index 000000000000..fede19b490fc --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java @@ -0,0 +1,163 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodList; +import io.fabric8.kubernetes.api.model.Quantity; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import org.apache.commons.lang.StringUtils; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@EnableKubernetesMockClient(crud = true) +class K8sTaskAdapterTest +{ + KubernetesClient client; + + private ObjectMapper jsonMapper; + + public K8sTaskAdapterTest() + { + TestUtils utils = new TestUtils(); + jsonMapper = utils.getTestObjectMapper(); + for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + } + jsonMapper.registerSubtypes( + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(IndexTask.IndexTuningConfig.class, "index") + ); + } + + @Test + void testAddingLabelsAndAnnotations() throws IOException + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig(); + config.namespace = "test"; + config.annotations.put("annotation_key", "annotation_value"); + config.labels.put("label_key", "label_value"); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter(testClient, config, jsonMapper); + Task task = K8sTestUtils.getTask(); + Job jobFromSpec = adapter.createJobFromPodSpec( + K8sTestUtils.getDummyPodSpec(), + task, + new PeonCommandContext(new ArrayList<>(), new ArrayList<>(), new File("/tmp/")) + ); + assertTrue(jobFromSpec.getMetadata().getAnnotations().containsKey("annotation_key")); + assertTrue(jobFromSpec.getMetadata().getAnnotations().containsKey(DruidK8sConstants.TASK_ID)); + assertFalse(jobFromSpec.getMetadata().getAnnotations().containsKey("label_key")); + assertTrue(jobFromSpec.getMetadata().getLabels().containsKey("label_key")); + assertTrue(jobFromSpec.getMetadata().getLabels().containsKey(DruidK8sConstants.LABEL_KEY)); + assertFalse(jobFromSpec.getMetadata().getLabels().containsKey("annotation_key")); + } + + @Test + public void serializingAndDeserializingATask() throws IOException + { + // given a task create a k8s job + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig(); + config.namespace = "test"; + K8sTaskAdapter adapter = new SingleContainerTaskAdapter(testClient, config, jsonMapper); + Task task = K8sTestUtils.getTask(); + Job jobFromSpec = adapter.createJobFromPodSpec( + K8sTestUtils.getDummyPodSpec(), + task, + new PeonCommandContext(new ArrayList<>(), new ArrayList<>(), new File("/tmp/")) + ); + + // cant launch jobs with test server, we have to hack around this. + Pod pod = K8sTestUtils.createPodFromJob(jobFromSpec); + client.pods().inNamespace("test").create(pod); + PodList podList = client.pods().inNamespace("test").list(); + assertEquals(1, podList.getItems().size()); + + // assert that the size of the pod is 1g + Pod myPod = Iterables.getOnlyElement(podList.getItems()); + Quantity containerMemory = myPod.getSpec().getContainers().get(0).getResources().getLimits().get("memory"); + String amount = containerMemory.getAmount(); + assertEquals(2400000000L, Long.valueOf(amount)); + assertTrue(StringUtils.isBlank(containerMemory.getFormat())); // no units specified we talk in bytes + + Task taskFromPod = adapter.toTask(Iterables.getOnlyElement(podList.getItems())); + assertEquals(task, taskFromPod); + } + + @Test + void testGrabbingTheLastXmxValueFromACommand() + { + List commands = Lists.newArrayList("-Xmx2g", "-Xms1g", "-Xmx4g"); + Optional value = K8sTaskAdapter.getJavaOptValueBytes("-Xmx", commands); + assertEquals(HumanReadableBytes.parse("4g"), value.get()); + + // one without Xmx + commands = new ArrayList<>(); + Optional result = K8sTaskAdapter.getJavaOptValueBytes("-Xmx", commands); + assertFalse(result.isPresent()); + } + + @Test + void testGettingContainerSize() + { + // nothing specified no heap no dbb should be (1g + 1g) * 1.2 + long expected = (long) ((HumanReadableBytes.parse("1g") + HumanReadableBytes.parse("1g")) * 1.2); + PeonCommandContext context = new PeonCommandContext( + new ArrayList<>(), + new ArrayList<>(), + new File("/tmp") + ); + assertEquals(expected, K8sTaskAdapter.getContainerMemory(context)); + + context = new PeonCommandContext( + new ArrayList<>(), + Collections.singletonList( + "-server -Xms512m -Xmx512m -XX:MaxDirectMemorySize=1g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=/druid/data -XX:+ExitOnOutOfMemoryError -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"), + new File("/tmp") + ); + expected = (long) ((HumanReadableBytes.parse("512m") + HumanReadableBytes.parse("1g")) * 1.2); + assertEquals(expected, K8sTaskAdapter.getContainerMemory(context)); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskIdTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskIdTest.java new file mode 100644 index 000000000000..c8579cd9b760 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskIdTest.java @@ -0,0 +1,35 @@ +/* + * 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.common; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class K8sTaskIdTest +{ + @Test + public void testModifyingTaskIDToBeK8sCompliant() + { + String original = "coordinator-issued_compact_k8smetrics_aeifmefd_2022-08-18T15:33:26.094Z"; + String result = new K8sTaskId(original).getK8sTaskId(); + assertEquals("coordinatorissuedcompactk8smetricsaeifmefd20220818t153326094z", result); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java new file mode 100644 index 000000000000..0f454102986d --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -0,0 +1,145 @@ +/* + * 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.common; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.PodSpecBuilder; +import io.fabric8.kubernetes.api.model.PodTemplateSpec; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.commons.text.CharacterPredicates; +import org.apache.commons.text.RandomStringGenerator; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; + +import java.io.File; + + +public class K8sTestUtils +{ + + private static final IndexSpec INDEX_SPEC = new IndexSpec(); + + + /* + * The k8s mock server can't launch pods from jobs, so we will fake it out by taking a job + * grabbing the podSpec and launching it ourselves for testing. + */ + @SuppressWarnings("javadoc") + public static Pod createPodFromJob(Job job) + { + RandomStringGenerator random = new RandomStringGenerator.Builder().withinRange('0', 'z') + .filteredBy(CharacterPredicates.LETTERS).build(); + PodTemplateSpec podTemplate = job.getSpec().getTemplate(); + return new PodBuilder() + .withNewMetadata() + .withName(new K8sTaskId(job.getMetadata().getName()).getK8sTaskId() + "-" + random.generate(5)) + .withLabels(ImmutableMap.of("job-name", new K8sTaskId(job.getMetadata().getName()).getK8sTaskId(), + DruidK8sConstants.LABEL_KEY, "true" + ) + ) + .endMetadata() + .withSpec(podTemplate.getSpec()) + .build(); + } + + public static PodSpec getDummyPodSpec() + { + return new PodSpecBuilder() + .addNewContainer() + .withName("pi") + .withImage("localhost:5000/busybox:stable") + .withCommand("perl", "-Mbignum=bpi", "-wle", "print bpi(2000)") + .endContainer() + .build(); + } + + public static Task getTask() + { + return new IndexTask( + null, + null, + new IndexTask.IndexIngestionSpec( + new DataSchema( + "foo", + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ), + null + ), + new IndexTask.IndexIOConfig( + null, + new LocalInputSource(new File("lol"), "rofl"), + new NoopInputFormat(), + true, + false + ), + new IndexTask.IndexTuningConfig( + null, + null, + null, + 10, + null, + null, + null, + 9999, + null, + null, + new DynamicPartitionsSpec(10000, null), + INDEX_SPEC, + null, + 3, + false, + null, + null, + null, + null, + null, + null, + null, + null, + 1L + ) + ), + null + ); + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.java new file mode 100644 index 000000000000..3f356b5f501f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.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.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; + +@EnableKubernetesMockClient(crud = true) +class MultiContainerTaskAdapterTest +{ + + KubernetesClient client; + + private ObjectMapper jsonMapper; + + public MultiContainerTaskAdapterTest() + { + TestUtils utils = new TestUtils(); + jsonMapper = utils.getTestObjectMapper(); + for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + } + jsonMapper.registerSubtypes( + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(IndexTask.IndexTuningConfig.class, "index") + ); + } + + @Test + public void testMultiContainerSupport() throws IOException + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + Pod pod = client.pods().load(this.getClass().getClassLoader().getResourceAsStream("multiContainerPodSpec.yaml")).get(); + KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig(); + config.namespace = "test"; + MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter(testClient, config, jsonMapper); + NoopTask task = NoopTask.create("id", 1); + Job actual = adapter.createJobFromPodSpec( + pod.getSpec(), + task, + new PeonCommandContext(Collections.singletonList("/peon.sh /druid/data/baseTaskDir/noop_2022-09-26T22:08:00.582Z_352988d2-5ff7-4b70-977c-3de96f9bfca6 1"), + new ArrayList<>(), + new File("/tmp") + ) + ); + Job expected = client.batch() + .v1() + .jobs() + .load(this.getClass().getClassLoader().getResourceAsStream("expectedMultiContainerOutput.yaml")) + .get(); + Assertions.assertEquals(expected, actual); + } + +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java new file mode 100644 index 000000000000..9c1e09ce87ca --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java @@ -0,0 +1,88 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; + +@EnableKubernetesMockClient(crud = true) +class SingleContainerTaskAdapterTest +{ + + KubernetesClient client; + + private ObjectMapper jsonMapper; + + public SingleContainerTaskAdapterTest() + { + TestUtils utils = new TestUtils(); + jsonMapper = utils.getTestObjectMapper(); + for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + } + jsonMapper.registerSubtypes( + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(IndexTask.IndexTuningConfig.class, "index") + ); + } + + @Test + public void testSingleContainerSupport() throws IOException + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + Pod pod = client.pods().load(this.getClass().getClassLoader().getResourceAsStream("multiContainerPodSpec.yaml")).get(); + KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig(); + config.namespace = "test"; + SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter(testClient, config, jsonMapper); + NoopTask task = NoopTask.create("id", 1); + Job actual = adapter.createJobFromPodSpec( + pod.getSpec(), + task, + new PeonCommandContext(Collections.singletonList("foo && bar"), + new ArrayList<>(), + new File("/tmp") + ) + ); + Job expected = client.batch() + .v1() + .jobs() + .load(this.getClass().getClassLoader().getResourceAsStream("expectedSingleiContainerOutput.yaml")) + .get(); + Assertions.assertEquals(expected, actual); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java new file mode 100644 index 000000000000..2cd1538cd727 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -0,0 +1,39 @@ +/* + * 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.common; + +import io.fabric8.kubernetes.client.KubernetesClient; + +public class TestKubernetesClient implements KubernetesClientApi +{ + + private final KubernetesClient client; + + public TestKubernetesClient(KubernetesClient client) + { + this.client = client; + } + + @Override + public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException + { + return executor.executeRequest(client); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml new file mode 100644 index 000000000000..157506fb1ded --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedMultiContainerOutput.yaml @@ -0,0 +1,108 @@ +apiVersion: "batch/v1" +kind: "Job" +metadata: + annotations: + task.id: "id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + name: "id" +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + template: + metadata: + annotations: + task.id: "id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + spec: + hostname: "id" + containers: + - args: + - "/kubexit/kubexit /bin/sh -c \"/peon.sh /druid/data/baseTaskDir/noop_2022-09-26T22:08:00.582Z_352988d2-5ff7-4b70-977c-3de96f9bfca6 1\"" + command: + - "/bin/sh" + - "-c" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_JSON" + value: "H4sIAAAAAAAAAEVOOw7CMAy9i+cOBYmlK0KItWVhNI0BSyEOToKoqt4doxZYLPv9/EbIQyRoIIhEqICd7TYquKqUePidDjN2UrSfxYEM0xKOfDdgvalr86aW0A0z9L9bSsVnc512nZkurHSTZJJQvK+gl5DpZfwIUVmU8wDNarJ0Ssu/EfCJ7PHM3tj9p9i3ltKjWKDbYsR+sU5vP86oMNUAAAA=" + - name: "JAVA_OPTS" + value: "" + - name: "druid_host" + valueFrom: + fieldRef: + fieldPath: "status.podIP" + - name: "HOSTNAME" + valueFrom: + fieldRef: + fieldPath: "metadata.name" + - name: "KUBEXIT_NAME" + value: "main" + - name: "KUBEXIT_GRAVEYARD" + value: "/graveyard" + + image: "one" + name: "main" + ports: + - containerPort: 8091 + name: "druid-tls-port" + protocol: "TCP" + - containerPort: 8100 + name: "druid-port" + protocol: "TCP" + resources: + limits: + cpu: "1000m" + memory: "2400000000" + requests: + cpu: "1000m" + memory: "2400000000" + volumeMounts: + - mountPath: "/graveyard" + name: "graveyard" + - mountPath: "/kubexit" + name: "kubexit" + - args: + - "/kubexit/kubexit /bin/sh -c \"/bin/sidekick -loggingEnabled=true -platform=platform\ + \ -splunkCluster=cluster -splunkIndexName=druid -splunkSourceType=json -splunkWorkingDir=/opt/splunkforwarder\ + \ -dataCenter=dc -environment=env -application=druid -instance=instance\ + \ -logFiles=/logs/druid/*.log\" || true" + command: + - "/bin/sh" + - "-c" + env: + - name: "KUBEXIT_NAME" + value: "sidecar" + - name: "KUBEXIT_GRAVEYARD" + value: "/graveyard" + - name: "KUBEXIT_DEATH_DEPS" + value: "main" + image: "two" + name: "sidecar" + volumeMounts: + - mountPath: "/graveyard" + name: "graveyard" + - mountPath: "/kubexit" + name: "kubexit" + initContainers: + - command: + - "cp" + - "/bin/kubexit" + - "/kubexit/kubexit" + image: "karlkfi/kubexit:v0.3.2" + name: "kubexit" + volumeMounts: + - mountPath: "/kubexit" + name: "kubexit" + restartPolicy: "Never" + volumes: + - emptyDir: + medium: "Memory" + name: "graveyard" + - emptyDir: {} + name: "kubexit" + ttlSecondsAfterFinished: 172800 \ No newline at end of file diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml new file mode 100644 index 000000000000..9b2b6a8e1f17 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedPodSpec.yaml @@ -0,0 +1,66 @@ +apiVersion: "batch/v1" +kind: "Job" +metadata: + annotations: + task.id: "task_id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + name: "taskid" +spec: + activeDeadlineSeconds: 3600 + backoffLimit: 0 + template: + metadata: + annotations: + task.id: "task_id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + spec: + containers: + - args: + - "trap 'touch /usr/share/pod/done' EXIT; mkdir -p ${TASK_DIR}; echo ${TASK_JSON}\ + \ | base64 -d | gzip -d > ${TASK_DIR}/task.json; " + command: + - "sh" + - "-c" + image: "one" + name: "main" + ports: + - containerPort: 8091 + name: "druid-tls-port" + protocol: "TCP" + - containerPort: 8100 + name: "druid-port" + protocol: "TCP" + resources: + limits: + cpu: "1000m" + memory: "1000000000" + requests: + cpu: "1000m" + memory: "1000000000" + volumeMounts: + - mountPath: "/usr/share/pod" + name: "peon-share" + - command: + - "tail -f /dev/null" + image: "two" + lifecycle: + postStart: + exec: + command: + - "while ! test -f /usr/share/pod/done; do echo 'Waiting for the main\ + \ pod to finish...'; sleep 5; done; echo 'Agent pod finished, exiting';\ + \ exit 0" + name: "sidecar" + volumeMounts: + - mountPath: "/usr/share/pod" + name: "peon-share" + readOnly: true + restartPolicy: "Never" + volumes: + - emptyDir: {} + name: "peon-share" + ttlSecondsAfterFinished: 7200 \ No newline at end of file diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleiContainerOutput.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleiContainerOutput.yaml new file mode 100644 index 000000000000..0650c0a29114 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedSingleiContainerOutput.yaml @@ -0,0 +1,60 @@ +apiVersion: "batch/v1" +kind: "Job" +metadata: + annotations: + task.id: "id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + name: "id" +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + template: + metadata: + annotations: + task.id: "id" + tls.enabled: "false" + labels: + druid.k8s.peons: "true" + spec: + hostname: "id" + containers: + - args: + - foo && bar + command: + - "sh" + - "-c" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_JSON" + value: "H4sIAAAAAAAAAEVOOw7CMAy9i+cOBYmlK0KItWVhNI0BSyEOToKoqt4doxZYLPv9/EbIQyRoIIhEqICd7TYquKqUePidDjN2UrSfxYEM0xKOfDdgvalr86aW0A0z9L9bSsVnc512nZkurHSTZJJQvK+gl5DpZfwIUVmU8wDNarJ0Ssu/EfCJ7PHM3tj9p9i3ltKjWKDbYsR+sU5vP86oMNUAAAA=" + - name: "JAVA_OPTS" + value: "" + - name: "druid_host" + valueFrom: + fieldRef: + fieldPath: "status.podIP" + - name: "HOSTNAME" + valueFrom: + fieldRef: + fieldPath: "metadata.name" + image: "one" + name: "main" + ports: + - containerPort: 8091 + name: "druid-tls-port" + protocol: "TCP" + - containerPort: 8100 + name: "druid-port" + protocol: "TCP" + resources: + limits: + cpu: "1000m" + memory: "2400000000" + requests: + cpu: "1000m" + memory: "2400000000" + restartPolicy: "Never" + ttlSecondsAfterFinished: 172800 \ No newline at end of file diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/multiContainerPodSpec.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/multiContainerPodSpec.yaml new file mode 100644 index 000000000000..ce2577e8db2f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/multiContainerPodSpec.yaml @@ -0,0 +1,26 @@ +apiVersion: v1 +kind: Pod +metadata: + name: test +spec: + containers: + - image: one + name: primary + command: + - "tail -f /dev/null" + - image: two + name: sidecar + args: + - -loggingEnabled=true + - -platform=platform + - -splunkCluster=cluster + - -splunkIndexName=druid + - -splunkSourceType=json + - -splunkWorkingDir=/opt/splunkforwarder + - -dataCenter=dc + - -environment=env + - -application=druid + - -instance=instance + - -logFiles=/logs/druid/*.log + command: + - /bin/sidekick diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/podSpec.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/podSpec.yaml new file mode 100644 index 000000000000..9f5e989076b1 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/podSpec.yaml @@ -0,0 +1,15 @@ +apiVersion: v1 +kind: Pod +metadata: + name: test +spec: + containers: + - command: + - sleep + - "3600" + image: one + name: primary + - command: + - "tail -f /dev/null" + image: two + name: sidecar diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index b1dc4545aafe..4bce2c90fa52 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -3071,7 +3071,8 @@ private void makeToolboxFactory() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); @@ -3195,7 +3196,9 @@ public void close() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 03b5be1bef8c..c59c479569f0 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -3035,7 +3035,8 @@ private void makeToolboxFactory() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); @@ -3160,7 +3161,9 @@ public void close() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index ff4c8c19ed05..31e77d7068ae 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -176,7 +176,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { final ServiceClientFactory clientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java index 5ce5b2290604..04eafc49cc42 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -98,7 +98,7 @@ public boolean isReady(final TaskActionClient taskActionClient) } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector); worker = new WorkerImpl(this, context); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java index 21de9f21244f..f38896fc14df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java @@ -333,6 +333,8 @@ protected FinalType submitRequest( if (location.equals(TaskLocation.unknown())) { throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", taskId)); } + log.info("Request Port is " + location.getPort()); + log.info("Request TLS Port is " + location.getTlsPort()); final Request request = createRequest( taskId, @@ -348,6 +350,7 @@ protected FinalType submitRequest( try { // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. + log.info("Request URL is" + request.getUrl()); checkConnection(request.getUrl().getHost(), request.getUrl().getPort()); response = submitRequest(request, responseHandler); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 551a4d1dcdec..bfea2d408a1b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -64,6 +64,7 @@ import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.tasklogs.TaskLogPusher; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -127,6 +128,10 @@ public class TaskToolbox private final ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private final ShuffleClient shuffleClient; + private final TaskLogPusher taskLogPusher; + private final String attemptId; + + public TaskToolbox( TaskConfig config, DruidNode taskExecutorNode, @@ -164,7 +169,9 @@ public TaskToolbox( OverlordClient overlordClient, CoordinatorClient coordinatorClient, ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, - ShuffleClient shuffleClient + ShuffleClient shuffleClient, + TaskLogPusher taskLogPusher, + String attemptId ) { this.config = config; @@ -205,6 +212,8 @@ public TaskToolbox( this.coordinatorClient = coordinatorClient; this.supervisorTaskClientProvider = supervisorTaskClientProvider; this.shuffleClient = shuffleClient; + this.taskLogPusher = taskLogPusher; + this.attemptId = attemptId; } public TaskConfig getConfig() @@ -461,6 +470,16 @@ public ShuffleClient getShuffleClient() return shuffleClient; } + public TaskLogPusher getTaskLogPusher() + { + return taskLogPusher; + } + + public String getAttemptId() + { + return attemptId; + } + public static class Builder { private TaskConfig config; @@ -500,6 +519,8 @@ public static class Builder private IntermediaryDataManager intermediaryDataManager; private ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private ShuffleClient shuffleClient; + private TaskLogPusher taskLogPusher; + private String attemptId; public Builder() { @@ -727,6 +748,18 @@ public Builder shuffleClient(final ShuffleClient shuffleClient) return this; } + public Builder taskLogPusher(final TaskLogPusher taskLogPusher) + { + this.taskLogPusher = taskLogPusher; + return this; + } + + public Builder attemptId(final String attemptId) + { + this.attemptId = attemptId; + return this; + } + public TaskToolbox build() { return new TaskToolbox( @@ -766,7 +799,9 @@ public TaskToolbox build() overlordClient, coordinatorClient, supervisorTaskClientProvider, - shuffleClient + shuffleClient, + taskLogPusher, + attemptId ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 7c0b8efee40a..9e8eae767089 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -30,6 +30,7 @@ import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.guice.annotations.AttemptId; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.RemoteChatHandler; @@ -60,6 +61,7 @@ import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.tasklogs.TaskLogPusher; import java.io.File; @@ -106,6 +108,8 @@ public class TaskToolboxFactory private final IntermediaryDataManager intermediaryDataManager; private final ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider; private final ShuffleClient shuffleClient; + private final TaskLogPusher taskLogPusher; + private final String attemptId; @Inject public TaskToolboxFactory( @@ -144,7 +148,9 @@ public TaskToolboxFactory( OverlordClient overlordClient, CoordinatorClient coordinatorClient, ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, - ShuffleClient shuffleClient + ShuffleClient shuffleClient, + TaskLogPusher taskLogPusher, + @AttemptId String attemptId ) { this.config = config; @@ -183,6 +189,8 @@ public TaskToolboxFactory( this.coordinatorClient = coordinatorClient; this.supervisorTaskClientProvider = supervisorTaskClientProvider; this.shuffleClient = shuffleClient; + this.taskLogPusher = taskLogPusher; + this.attemptId = attemptId; } public TaskToolbox build(Task task) @@ -230,6 +238,8 @@ public TaskToolbox build(Task task) .coordinatorClient(coordinatorClient) .supervisorTaskClientProvider(supervisorTaskClientProvider) .shuffleClient(shuffleClient) + .taskLogPusher(taskLogPusher) + .attemptId(attemptId) .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index a1db708c4d37..ce2aa2dd08ce 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -118,6 +118,9 @@ public enum BatchProcessingMode @JsonProperty private final boolean storeEmptyColumns; + @JsonProperty + private final boolean enableTaskLevelLogPush; + @JsonCreator public TaskConfig( @JsonProperty("baseDir") String baseDir, @@ -132,7 +135,8 @@ public TaskConfig( @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, @JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMappedIndex, // deprecated, only set to true to fall back to older behavior @JsonProperty("batchProcessingMode") String batchProcessingMode, - @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns + @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns, + @JsonProperty("enableTaskLevelLogPush") boolean enableTaskLevelLogPush ) { this.baseDir = baseDir == null ? System.getProperty("java.io.tmpdir") : baseDir; @@ -160,6 +164,8 @@ public TaskConfig( this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; this.batchMemoryMappedIndex = batchMemoryMappedIndex; + + this.enableTaskLevelLogPush = enableTaskLevelLogPush; // Conflict resolution. Assume that if batchMemoryMappedIndex is set (since false is the default) that // the user changed it intentionally to use legacy, in this case oveeride batchProcessingMode and also // set it to legacy else just use batchProcessingMode and don't pay attention to batchMemoryMappedIndexMode: @@ -280,6 +286,12 @@ public boolean isStoreEmptyColumns() return storeEmptyColumns; } + @JsonProperty + public boolean isEnableTaskLevelLogPush() + { + return enableTaskLevelLogPush; + } + private String defaultDir(@Nullable String configParameter, final String defaultVal) { if (configParameter == null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 8fcadc301d22..1c9508856b02 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -31,7 +31,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; @@ -155,7 +154,7 @@ protected AbstractBatchIndexTask( * @see #stopGracefully(TaskConfig) */ @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public String setup(TaskToolbox toolbox) throws Exception { if (taskLockHelper == null) { // Subclasses generally use "isReady" to initialize the taskLockHelper. It's not guaranteed to be called before @@ -170,8 +169,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception synchronized (this) { if (stopped) { - String errMsg = "Attempting to run a task that has been stopped. See overlord & task logs for more details."; - return TaskStatus.failure(getId(), errMsg); + return "Attempting to run a task that has been stopped. See overlord & task logs for more details."; } else { // Register the cleaner to interrupt the current thread first. // Since the resource closer cleans up the registered resources in LIFO order, @@ -184,7 +182,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception resourceCloserOnAbnormalExit.register(config -> currentThread.interrupt()); } } - return runTask(toolbox); + return super.setup(toolbox); } @Override @@ -250,11 +248,6 @@ protected void registerResourceCloserOnAbnormalExit(Consumer cleaner } } - /** - * The method to actually process this task. This method is executed in {@link #run(TaskToolbox)}. - */ - public abstract TaskStatus runTask(TaskToolbox toolbox) throws Exception; - /** * Return true if this task can overwrite existing segments. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java index 094ae1de824f..05c8cef470c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java @@ -27,10 +27,13 @@ import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.Query; @@ -40,7 +43,9 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; +import java.nio.file.Paths; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,6 +53,8 @@ public abstract class AbstractTask implements Task { + private static final Logger log = new Logger(AbstractTask.class); + // This is mainly to avoid using combinations of IOConfig flags to figure out the ingestion mode and // also to use the mode as dimension in metrics public enum IngestionMode @@ -82,6 +89,8 @@ public static IngestionMode fromString(String name) private final String dataSource; private final Map context; + private File reportsFile; + private File logFile; private final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); @@ -125,6 +134,50 @@ protected AbstractTask( this(id, groupId, taskResource, dataSource, context, IngestionMode.NONE); } + @Nullable + public String setup(TaskToolbox toolbox) throws Exception + { + File taskDir = toolbox.getConfig().getTaskDir(getId()); + FileUtils.mkdirp(taskDir); + File attemptDir = Paths.get(taskDir.getAbsolutePath(), "attempt", toolbox.getAttemptId()).toFile(); + FileUtils.mkdirp(attemptDir); + File statusFile = new File(attemptDir, "status.json"); + reportsFile = new File(attemptDir, "report.json"); + log.debug("Task setup complete"); + return null; + } + + @Override + public final TaskStatus run(TaskToolbox taskToolbox) throws Exception + { + try { + String errorMessage = setup(taskToolbox); + if (org.apache.commons.lang3.StringUtils.isNotBlank(errorMessage)) { + return TaskStatus.failure(getId(), errorMessage); + } + return runTask(taskToolbox); + } + finally { + cleanUp(taskToolbox); + } + } + + public abstract TaskStatus runTask(TaskToolbox taskToolbox) throws Exception; + + public void cleanUp(TaskToolbox toolbox) throws Exception + { + if (toolbox.getConfig().isEnableTaskLevelLogPush()) { + if (reportsFile != null && reportsFile.exists()) { + toolbox.getTaskLogPusher().pushTaskReports(id, reportsFile); + log.debug("Pushed task reports"); + } else { + log.debug("No task reports file exists to push"); + } + } else { + log.debug("Not pushing task logs and reports from task."); + } + } + public static String getOrMakeId(@Nullable String id, final String typeName, String dataSource) { return getOrMakeId(id, typeName, dataSource, null); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 7eb60b27ccbd..ca7fadb24f91 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -242,7 +242,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - public TaskStatus run(final TaskToolbox toolbox) + public TaskStatus runTask(final TaskToolbox toolbox) { runThread = Thread.currentThread(); authorizerMapper = toolbox.getAuthorizerMapper(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java index 4e048ad4d2bf..d6379f4ee680 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java @@ -60,7 +60,7 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final TaskLock myLock = getAndCheckLock(toolbox); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java index d186a9770dde..1b4f1776d8db 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java @@ -89,7 +89,7 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final NavigableMap> taskLockMap = getTaskLockMap(toolbox.getTaskActionClient()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java index 152eb33043b6..3065f69572c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java @@ -68,7 +68,7 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final TaskLock myLock = getAndCheckLock(toolbox); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java index aa7efbbcb1f0..bce3eb41839d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java @@ -145,7 +145,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { if (firehoseFactory != null) { log.info("Connecting firehose"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 65c578c49a0c..9c3c5cdb2380 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -208,7 +208,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { runThread = Thread.currentThread(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java index 622c9ff8b018..51a101582157 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java @@ -61,7 +61,7 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final TaskLock myLock = getAndCheckLock(toolbox); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index db325f8778dc..d8130892d5ae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -226,7 +226,7 @@ public String getSubtaskSpecId() } @Override - public TaskStatus runTask(final TaskToolbox toolbox) + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { try { if (missingIntervalsInOverwriteMode) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index 98ef0739d21f..0e375fffb234 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -22,11 +22,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.CharMatcher; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -78,14 +75,15 @@ import java.io.InputStream; import java.io.OutputStream; import java.math.RoundingMode; +import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; -import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -154,9 +152,10 @@ public ListenableFuture run(final Task task) @Override public TaskStatus call() { - final String attemptUUID = UUID.randomUUID().toString(); + + final String attemptId = String.valueOf(getNextAttemptID(taskConfig, task.getId())); final File taskDir = taskConfig.getTaskDir(task.getId()); - final File attemptDir = new File(taskDir, attemptUUID); + final File attemptDir = Paths.get(taskDir.getAbsolutePath(), "attempt", attemptId).toFile(); final ProcessHolder processHolder; final String childHost = node.getHost(); @@ -176,8 +175,6 @@ public TaskStatus call() try { final Closer closer = Closer.create(); try { - FileUtils.mkdirp(attemptDir); - final File taskFile = new File(taskDir, "task.json"); final File statusFile = new File(attemptDir, "status.json"); final File logFile = new File(taskDir, "log"); @@ -304,6 +301,15 @@ public TaskStatus call() } } + // add the attemptId as a system property + command.add( + StringUtils.format( + "-D%s=%s", + "attemptId", + "1" + ) + ); + // Add dataSource, taskId and taskType for metrics or logging command.add( StringUtils.format( @@ -363,9 +369,8 @@ public TaskStatus call() command.add("org.apache.druid.cli.Main"); command.add("internal"); command.add("peon"); - command.add(taskFile.toString()); - command.add(statusFile.toString()); - command.add(reportsFile.toString()); + command.add(taskDir.toString()); + command.add(attemptId); String nodeType = task.getNodeType(); if (nodeType != null) { command.add("--nodeType"); @@ -512,7 +517,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo } finally { Thread.currentThread().setName(priorThreadName); - // Upload task logs + // Upload task logs taskLogPusher.pushTaskLog(task.getId(), logFile); if (reportsFile.exists()) { taskLogPusher.pushTaskReports(task.getId(), reportsFile); @@ -685,7 +690,7 @@ private void shutdownTaskProcess(ForkingTaskRunnerWorkItem taskInfo) } } - String getMaskedCommand(List maskedProperties, List command) + public static String getMaskedCommand(List maskedProperties, List command) { final Set maskedPropertiesSet = Sets.newHashSet(maskedProperties); final Iterator maskedIterator = command.stream().map(element -> { @@ -879,40 +884,31 @@ void shutdown() process.destroy(); } } -} - -/** - * Make an iterable of space delimited strings... unless there are quotes, which it preserves - */ -class QuotableWhiteSpaceSplitter implements Iterable -{ - private final String string; - public QuotableWhiteSpaceSplitter(String string) + @VisibleForTesting + static int getNextAttemptID(TaskConfig config, String taskId) { - this.string = Preconditions.checkNotNull(string); - } - - @Override - public Iterator iterator() - { - return Splitter.on( - new CharMatcher() - { - private boolean inQuotes = false; - - @Override - public boolean matches(char c) - { - if ('"' == c) { - inQuotes = !inQuotes; - } - if (inQuotes) { - return false; - } - return CharMatcher.BREAKING_WHITESPACE.matches(c); - } - } - ).omitEmptyStrings().split(string).iterator(); + File taskDir = config.getTaskDir(taskId); + File attemptDir = new File(taskDir, "attempt"); + try { + FileUtils.mkdirp(attemptDir); + } + catch (IOException e) { + throw new ISE("Error creating directory", e); + } + int maxAttempt = + Arrays.stream(attemptDir.listFiles(File::isDirectory)) + .mapToInt(x -> Integer.parseInt(x.getName())) + .max().orElse(0); + // now make the directory + File attempt = new File(attemptDir, String.valueOf(maxAttempt + 1)); + try { + FileUtils.mkdirp(attempt); + } + catch (IOException e) { + throw new ISE("Error creating directory", e); + } + return maxAttempt + 1; } } + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/QuotableWhiteSpaceSplitter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/QuotableWhiteSpaceSplitter.java new file mode 100644 index 000000000000..de0a04390b17 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/QuotableWhiteSpaceSplitter.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +import com.google.common.base.CharMatcher; +import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; + +import java.util.Iterator; + +/** + * Make an iterable of space delimited strings... unless there are quotes, which it preserves + */ +public class QuotableWhiteSpaceSplitter implements Iterable +{ + private final String string; + + public QuotableWhiteSpaceSplitter(String string) + { + this.string = Preconditions.checkNotNull(string); + } + + @Override + public Iterator iterator() + { + return Splitter.on( + new CharMatcher() + { + private boolean inQuotes = false; + + @Override + public boolean matches(char c) + { + if ('"' == c) { + inQuotes = !inQuotes; + } + if (inQuotes) { + return false; + } + return CharMatcher.BREAKING_WHITESPACE.matches(c); + } + } + ).omitEmptyStrings().split(string).iterator(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java index 5cbdabb3edad..f5ce895709ee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java @@ -135,4 +135,12 @@ default TaskLocation getTaskLocation(String taskId) Map getLazyTaskSlotCount(); Map getBlacklistedTaskSlotCount(); + + /** + * Beacause the k8s task runner is an extension, we need to know the task runner type in the overlord resource + */ + default boolean isK8sTaskRunner() + { + return false; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index a3a7f8de69d9..72dcdcaa6a47 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -31,6 +31,8 @@ import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.IndexingWorker; +import org.apache.druid.client.indexing.IndexingWorkerInfo; import org.apache.druid.common.config.ConfigManager.SetResult; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.RunnerTaskState; @@ -339,7 +341,9 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) taskInfo.getStatus().getStatusCode(), RunnerTaskState.WAITING, taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation() == null ? TaskLocation.unknown() : taskInfo.getStatus().getLocation(), + taskInfo.getStatus().getLocation() == null + ? TaskLocation.unknown() + : taskInfo.getStatus().getLocation(), taskInfo.getDataSource(), taskInfo.getStatus().getErrorMsg() ) @@ -502,9 +506,10 @@ public Response getTotalWorkerCapacity() } } else { // Auto scale is not using DefaultWorkerBehaviorConfig - log.debug("Cannot calculate maximum worker capacity as WorkerBehaviorConfig [%s] of type [%s] does not support getting max capacity", - workerBehaviorConfig, - workerBehaviorConfig.getClass().getSimpleName() + log.debug( + "Cannot calculate maximum worker capacity as WorkerBehaviorConfig [%s] of type [%s] does not support getting max capacity", + workerBehaviorConfig, + workerBehaviorConfig.getClass().getSimpleName() ); maximumCapacity = -1; } @@ -927,6 +932,23 @@ public Response apply(TaskRunner taskRunner) { if (taskRunner instanceof WorkerTaskRunner) { return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build(); + } else if (taskRunner.isK8sTaskRunner()) { + return Response.ok(ImmutableList.of( + new IndexingWorkerInfo( + new IndexingWorker( + "http", + "host", + "8100", + taskRunner.getTotalTaskSlotCount().getOrDefault("taskQueue", 0L).intValue(), + "version" + ), + 0, + Collections.emptySet(), + Collections.emptyList(), + DateTimes.EPOCH, + null + ) + )).build(); } else { log.debug( "Task runner [%s] of type [%s] does not support listing workers", diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index f8502bac2657..302099685b4a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -145,7 +145,7 @@ public SeekableStreamIndexTaskIOConfig getI } @Override - public TaskStatus run(final TaskToolbox toolbox) + public TaskStatus runTask(final TaskToolbox toolbox) { emitMetric(toolbox.getEmitter(), "ingest/count", 1); return getRunner().run(toolbox); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java index 7d5d5650ef86..b38ac0e2e898 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -104,69 +104,65 @@ public void start() throws InterruptedException throw new RuntimeException(e); } - // Avoid running the same task twice on the same machine by locking the task base directory. - - final File taskLockFile = taskConfig.getTaskLockFile(task.getId()); + // if the parent stream is not defined, like running a k8s peon task, it will never run on the same + // pod twice, no need to lock. + if (taskExecutorConfig.isParentStreamDefined()) { + // Avoid running the same task twice on the same machine by locking the task base directory. + final File taskLockFile = taskConfig.getTaskLockFile(task.getId()); + try { + synchronized (this) { + if (taskLockChannel == null && taskLockFileLock == null) { + taskLockChannel = FileChannel.open( + taskLockFile.toPath(), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE + ); + + log.info("Attempting to lock file[%s].", taskLockFile); + final long startLocking = System.currentTimeMillis(); + final long timeout = DateTimes.utc(startLocking).plus(taskConfig.getDirectoryLockTimeout()).getMillis(); + while (taskLockFileLock == null && System.currentTimeMillis() < timeout) { + taskLockFileLock = taskLockChannel.tryLock(); + if (taskLockFileLock == null) { + Thread.sleep(100); + } + } - try { - synchronized (this) { - if (taskLockChannel == null && taskLockFileLock == null) { - taskLockChannel = FileChannel.open( - taskLockFile.toPath(), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE - ); - - log.info("Attempting to lock file[%s].", taskLockFile); - final long startLocking = System.currentTimeMillis(); - final long timeout = DateTimes.utc(startLocking).plus(taskConfig.getDirectoryLockTimeout()).getMillis(); - while (taskLockFileLock == null && System.currentTimeMillis() < timeout) { - taskLockFileLock = taskLockChannel.tryLock(); if (taskLockFileLock == null) { - Thread.sleep(100); + throw new ISE("Could not acquire lock file[%s] within %,dms.", taskLockFile, timeout - startLocking); + } else { + log.info("Acquired lock file[%s] in %,dms.", taskLockFile, System.currentTimeMillis() - startLocking); } - } - - if (taskLockFileLock == null) { - throw new ISE("Could not acquire lock file[%s] within %,dms.", taskLockFile, timeout - startLocking); } else { - log.info("Acquired lock file[%s] in %,dms.", taskLockFile, System.currentTimeMillis() - startLocking); + throw new ISE("Already started!"); } - } else { - throw new ISE("Already started!"); } } - } - catch (IOException e) { - throw new RuntimeException(e); - } + catch (IOException e) { + throw new RuntimeException(e); + } - if (taskExecutorConfig.isParentStreamDefined()) { // Spawn monitor thread to keep a watch on parent's stdin // If stdin reaches eof, the parent is gone, and we should shut down parentMonitorExec.submit( - new Runnable() - { - @Override - public void run() - { - try { - while (parentStream.read() != -1) { - // Toss the byte - } - } - catch (Exception e) { - log.error(e, "Failed to read from stdin"); + () -> { + try { + while (parentStream.read() != -1) { + // Toss the byte } - - // Kind of gross, but best way to kill the JVM as far as I know - log.info("Triggering JVM shutdown."); - System.exit(2); } + catch (Exception e) { + log.error(e, "Failed to read from stdin"); + } + + // Kind of gross, but best way to kill the JVM as far as I know + log.info("Triggering JVM shutdown."); + System.exit(2); } ); } + // Won't hurt in remote mode, and is required for setting up locks in local mode: try { if (!task.isReady(taskActionClientFactory.create(task))) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java index 4e24431576ee..0c48fe299a82 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -75,6 +75,12 @@ public ExecutorLifecycleConfig setStatusFile(File statusFile) return this; } + public ExecutorLifecycleConfig setParentStreamDefined(boolean parentStreamDefined) + { + this.parentStreamDefined = parentStreamDefined; + return this; + } + public InputStream getParentStream() { if ("stdin".equals(parentStreamName)) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 3e3129900384..af9371928915 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -120,7 +120,8 @@ public void setUp() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false), mockTaskActionClientFactory, @@ -156,7 +157,9 @@ public void setUp() throws IOException new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java index e8975e39ff43..eca5ed3557a1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java @@ -75,7 +75,7 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { return status; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java index cb191e271f98..fc7a6c991566 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java @@ -75,7 +75,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { return TaskStatus.success(getId()); } @@ -108,7 +108,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { while (!Thread.currentThread().isInterrupted()) { Thread.sleep(1000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index ecae743374f8..f2a742bdb250 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1565,7 +1565,8 @@ public SegmentPublishResult announceHistoricalSegments( false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( @@ -1657,7 +1658,9 @@ public void close() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index f378a53a4dd2..786ff21255fd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -581,7 +581,8 @@ private static TaskToolbox makeTaskToolbox( false, false, mode.name(), - null + null, + false ) ) .joinableFactory(NoopJoinableFactory.INSTANCE) @@ -592,7 +593,10 @@ private static TaskToolbox makeTaskToolbox( .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) .chatHandlerProvider(new NoopChatHandlerProvider()) .appenderatorsManager(new TestAppenderatorsManager()) + .taskLogPusher(null) + .attemptId("1") .build(); + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index b0ddf61c5602..500d49b0d807 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1696,7 +1696,8 @@ public List getLocations() false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ) ) .taskActionClient(createActionClient(task)) @@ -1715,6 +1716,8 @@ public List getLocations() .appenderatorsManager(new TestAppenderatorsManager()) .overlordClient(overlordClient) .coordinatorClient(coordinatorClient) + .taskLogPusher(null) + .attemptId("1") .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 69a17e6128f4..458733d6ab2e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -2030,7 +2030,8 @@ public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ) ) .taskActionClient(taskActionClient) @@ -2049,6 +2050,8 @@ public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) .appenderatorsManager(new TestAppenderatorsManager()) .coordinatorClient(COORDINATOR_CLIENT) .segmentCacheManager(segmentCacheManager) + .taskLogPusher(null) + .attemptId("1") .build(); } @@ -2344,7 +2347,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { throw new UnsupportedOperationException(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index 5a9ae7f2ceb8..9d62b3ab3fe6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -119,7 +119,8 @@ public TaskStatus runTask(TaskToolbox toolbox) false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false )).once(); EasyMock.replay(toolbox); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index c6d10f2b87e3..2d3cb8dbad53 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -328,7 +328,8 @@ public ListenableFuture run(Task task) false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ) ) .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) @@ -346,8 +347,11 @@ public ListenableFuture run(Task task) .chatHandlerProvider(new NoopChatHandlerProvider()) .rowIngestionMetersFactory(testUtils.getRowIngestionMetersFactory()) .appenderatorsManager(new TestAppenderatorsManager()) + .taskLogPusher(null) + .attemptId("1") .build(); + if (task.isReady(box.getTaskActionClient())) { return Futures.immediateFuture(task.run(box)); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 01232c3875a9..d571bd0bc47d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -899,7 +899,8 @@ private TaskToolbox makeToolbox( false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, mdc); try { @@ -1009,7 +1010,9 @@ public void close() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index b72e39a657d9..b7cc3159ff26 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -257,7 +257,8 @@ public void setUpAbstractParallelIndexSupervisorTaskTest() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), null ); @@ -657,7 +658,8 @@ public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); objectMapper.setInjectableValues( @@ -707,7 +709,8 @@ protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ) ) .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) @@ -741,6 +744,8 @@ public File getStorageDirectory() .coordinatorClient(coordinatorClient) .supervisorTaskClientProvider(new LocalParallelIndexTaskClientProvider(taskRunner, transientApiCallFailureRate)) .shuffleClient(new LocalShuffleClient(intermediaryDataManager)) + .taskLogPusher(null) + .attemptId("1") .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunnerTest.java index 61f3bd74ae24..b09841e916b8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunnerTest.java @@ -240,9 +240,9 @@ private ReportingNoopTask(String groupId, TestPhaseRunner phaseRunner) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { - final TaskStatus result = super.run(toolbox); + final TaskStatus result = super.runTask(toolbox); phaseRunner.collectReport(new EmptySubTaskReport(getId())); return result; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 199a6fad88fe..7bf894fff4d7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -404,7 +404,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { final TestInputSource inputSource = (TestInputSource) getIngestionSchema().getIOConfig().getInputSource(); final TestInput testInput = Iterables.getOnlyElement(inputSource.splits).get(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 6e00b14c670c..b11d939f760f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -662,7 +662,7 @@ private class TestSubTask extends SinglePhaseSubTask } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) throws Exception { while (state == TaskState.RUNNING) { Thread.sleep(100); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java index caff8cc81d77..5150660aba5e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java @@ -40,6 +40,7 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -235,15 +236,28 @@ private class TestTask extends NoopTask this.throwUnknownTypeIdError = throwUnknownTypeIdError; } + @Nullable @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public String setup(TaskToolbox toolbox) throws Exception + { + return null; + } + + @Override + public void cleanUp(TaskToolbox toolbox) throws Exception + { + // do nothing + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { monitor.collectReport(new SimpleSubTaskReport(getId())); if (shouldFail) { Thread.sleep(getRunTime()); return TaskStatus.failure(getId(), "Dummy task status failure for testing"); } else { - return super.run(toolbox); + return super.runTask(toolbox); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java index 2d1a791586a1..9b64064efe28 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java @@ -40,21 +40,28 @@ import org.assertj.core.util.Lists; import org.joda.time.Period; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; import java.io.File; import java.io.IOException; +import java.nio.file.Paths; import java.util.List; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; +import static org.junit.Assert.assertEquals; + public class ForkingTaskRunnerTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); // This tests the test to make sure the test fails when it should. @Test(expected = AssertionError.class) @@ -72,7 +79,7 @@ public void testPatternMatcherFailureForSpaceOnlyJavaOptions() @Test public void testPatternMatcherLeavesUnbalancedQuoteJavaOptions() { - Assert.assertEquals("\"", Iterators.get(new QuotableWhiteSpaceSplitter("\"").iterator(), 0)); + assertEquals("\"", Iterators.get(new QuotableWhiteSpaceSplitter("\"").iterator(), 0)); } @Test @@ -124,7 +131,7 @@ public void testEmpty() @Test public void testFarApart() { - Assert.assertEquals( + assertEquals( ImmutableList.of("start", "stop"), ImmutableList.copyOf( new QuotableWhiteSpaceSplitter( "start\t\t\t\t \n\f\r\n \f\f \n\r\f\n\r\t stop" @@ -145,7 +152,7 @@ public void testOmitEmpty() private static void checkValues(String[] strings) { - Assert.assertEquals( + assertEquals( ImmutableList.copyOf(strings), ImmutableList.copyOf(new QuotableWhiteSpaceSplitter(Joiner.on(" ").join(strings))) ); @@ -182,7 +189,7 @@ public void testMaskedIterator() null, startupLoggingConfig ); - Assert.assertEquals( + assertEquals( originalAndExpectedCommand.rhs, forkingTaskRunner.getMaskedCommand( startupLoggingConfig.getMaskProperties(), @@ -209,7 +216,8 @@ public void testTaskStatusWhenTaskProcessFails() throws ExecutionException, Inte false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new Properties(), @@ -244,8 +252,8 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo forkingTaskRunner.setNumProcessorsPerTask(); final TaskStatus status = forkingTaskRunner.run(NoopTask.create()).get(); - Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); - Assert.assertEquals( + assertEquals(TaskState.FAILED, status.getStatusCode()); + assertEquals( "Task execution process exited unsuccessfully with code[1]. See middleManager logs for more details.", status.getErrorMsg() ); @@ -254,15 +262,16 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo } @Test - public void testTaskStatusWhenTaskProcessSucceedsTaskSucceeds() throws ExecutionException, InterruptedException + public void testTaskStatusWhenTaskProcessSucceedsTaskSucceeds() throws Exception { ObjectMapper mapper = new DefaultObjectMapper(); Task task = NoopTask.create(); + File file = temporaryFolder.newFolder(); ForkingTaskRunner forkingTaskRunner = new ForkingTaskRunner( new ForkingTaskRunnerConfig(), new TaskConfig( null, - null, + file.toString(), null, null, ImmutableList.of(), @@ -273,7 +282,8 @@ public void testTaskStatusWhenTaskProcessSucceedsTaskSucceeds() throws Execution false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new Properties(), @@ -291,8 +301,9 @@ ProcessHolder runTaskProcess(List command, File logFile, TaskLocation ta Mockito.doNothing().when(processHolder).shutdown(); for (String param : command) { - if (param.endsWith("status.json")) { - mapper.writeValue(new File(param), TaskStatus.success(task.getId())); + if (param.endsWith(task.getId())) { + File resultFile = Paths.get(taskConfig.getTaskDir(task.getId()).getAbsolutePath(), "attempt", "1", "status.json").toFile(); + mapper.writeValue(resultFile, TaskStatus.success(task.getId())); break; } } @@ -315,22 +326,23 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo forkingTaskRunner.setNumProcessorsPerTask(); final TaskStatus status = forkingTaskRunner.run(task).get(); - Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); + assertEquals(TaskState.SUCCESS, status.getStatusCode()); Assert.assertNull(status.getErrorMsg()); Assert.assertEquals(0L, (long) forkingTaskRunner.getWorkerFailedTaskCount()); Assert.assertEquals(1L, (long) forkingTaskRunner.getWorkerSuccessfulTaskCount()); } @Test - public void testTaskStatusWhenTaskProcessSucceedsTaskFails() throws ExecutionException, InterruptedException + public void testTaskStatusWhenTaskProcessSucceedsTaskFails() throws Exception { ObjectMapper mapper = new DefaultObjectMapper(); Task task = NoopTask.create(); + File file = temporaryFolder.newFolder(); ForkingTaskRunner forkingTaskRunner = new ForkingTaskRunner( new ForkingTaskRunnerConfig(), new TaskConfig( null, - null, + file.toString(), null, null, ImmutableList.of(), @@ -341,7 +353,8 @@ public void testTaskStatusWhenTaskProcessSucceedsTaskFails() throws ExecutionExc false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new Properties(), @@ -359,8 +372,9 @@ ProcessHolder runTaskProcess(List command, File logFile, TaskLocation ta Mockito.doNothing().when(processHolder).shutdown(); for (String param : command) { - if (param.endsWith("status.json")) { - mapper.writeValue(new File(param), TaskStatus.failure(task.getId(), "task failure test")); + if (param.endsWith(task.getId())) { + File resultFile = Paths.get(taskConfig.getTaskDir(task.getId()).getAbsolutePath(), "attempt", "1", "status.json").toFile(); + mapper.writeValue(resultFile, TaskStatus.failure(task.getId(), "task failure test")); break; } } @@ -377,8 +391,34 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo forkingTaskRunner.setNumProcessorsPerTask(); final TaskStatus status = forkingTaskRunner.run(task).get(); - Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); - Assert.assertEquals("task failure test", status.getErrorMsg()); + assertEquals(TaskState.FAILED, status.getStatusCode()); + assertEquals("task failure test", status.getErrorMsg()); + } + + @Test + public void testGettingTheNextAttemptDir() throws IOException + { + File file = temporaryFolder.newFolder(); + TaskConfig taskConfig = new TaskConfig( + null, + file.toString(), + null, + null, + ImmutableList.of(), + false, + new Period("PT0S"), + new Period("PT10S"), + ImmutableList.of(), + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), + null, + false + ); + String taskId = "foo"; + assertEquals(1, ForkingTaskRunner.getNextAttemptID(taskConfig, taskId)); + assertEquals(2, ForkingTaskRunner.getNextAttemptID(taskConfig, taskId)); + assertEquals(3, ForkingTaskRunner.getNextAttemptID(taskConfig, taskId)); } @Test @@ -418,7 +458,8 @@ public void testJavaOptsAndJavaOptsArrayOverride() throws ExecutionException, In false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new Properties(), @@ -484,7 +525,8 @@ public void testInvalidTaskContextJavaOptsArray() throws JsonProcessingException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new Properties(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index 7b74ac048049..d2c3e7eecb59 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -67,7 +67,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final Interval interval1 = Intervals.of("2010-01-01T00/PT1H"); final Interval interval2 = Intervals.of("2010-01-01T01/PT1H"); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 0278dca8bc16..a79efbcec2ce 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -98,7 +98,8 @@ public void setup() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final ServiceEmitter emitter = new NoopServiceEmitter(); EmittingLogger.registerEmitter(emitter); @@ -138,7 +139,9 @@ public void setup() throws IOException new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); runner = new SingleTaskBackgroundRunner( toolboxFactory, @@ -239,7 +242,7 @@ public void statusChanged(String taskId, TaskStatus status) new RestorableTask(new BooleanHolder()) { @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { throw new Error("task failure test"); } @@ -336,7 +339,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { return TaskStatus.success(getId()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 8d4e2966486d..325ebc1755a0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -616,7 +616,8 @@ private TaskToolboxFactory setUpTaskToolboxFactory( false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); return new TaskToolboxFactory( @@ -700,7 +701,9 @@ public void unannounceSegments(Iterable segments) new NoopOverlordClient(), null, null, - null + null, + null, + "1" ); } @@ -1035,7 +1038,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final Interval interval = Intervals.of("2012-01-01/P1D"); final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction( @@ -1085,7 +1088,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); @@ -1127,7 +1130,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); @@ -1464,7 +1467,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus runTask(TaskToolbox toolbox) throws Exception { final Interval interval = Intervals.of("2012-01-01/P1D"); final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 1da1ae18f84d..11f241f43d6d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -1378,7 +1378,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { return TaskStatus.failure("how?", "Dummy task status err msg"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index a90ad8b3c090..2abfb030b8b3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -372,6 +372,19 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception return tryTimeChunkLock(taskActionClient, ImmutableList.of(interval)); } + @Override + public String setup(TaskToolbox toolbox) throws Exception + { + // do nothing + return null; + } + + @Override + public void cleanUp(TaskToolbox toolbox) throws Exception + { + // do nothing + } + @Override public TaskStatus runTask(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java index fd89cb047be7..4a3c478873b5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java @@ -63,7 +63,8 @@ public void testTaskStatusWhenTaskThrowsExceptionWhileRunning() throws Execution false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ), new WorkerConfig(), new NoopTaskLogs(), @@ -93,7 +94,7 @@ public void stopGracefully(TaskConfig taskConfig) } @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { throw new RuntimeException("Task failure test"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 80c734e0bfe6..518c1a57cf49 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -95,7 +95,8 @@ private WorkerTaskManager createWorkerTaskManager() false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class); @@ -142,7 +143,9 @@ private WorkerTaskManager createWorkerTaskManager() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ), taskConfig, location @@ -274,7 +277,7 @@ public void testTaskStatusWhenTaskRunnerFutureThrowsException() throws Exception Task task = new NoopTask("id", null, null, 100, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)) { @Override - public TaskStatus run(TaskToolbox toolbox) + public TaskStatus runTask(TaskToolbox toolbox) { throw new Error("task failure test"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index b6261db8983e..f2085cf88b8f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -166,7 +166,8 @@ private WorkerTaskMonitor createTaskMonitor() false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class); @@ -212,7 +213,9 @@ private WorkerTaskMonitor createTaskMonitor() new NoopOverlordClient(), null, null, - null + null, + null, + "1" ), taskConfig, new NoopServiceEmitter(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfigTest.java new file mode 100644 index 000000000000..8dfcfa2b49b6 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfigTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.worker.executor; + +import org.junit.Assert; +import org.junit.Test; + + +public class ExecutorLifecycleConfigTest +{ + @Test + public void executorLifecycleConfigTest() + { + ExecutorLifecycleConfig config = new ExecutorLifecycleConfig(); + config.setParentStreamDefined(false); + Assert.assertFalse(config.isParentStreamDefined()); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java index 79b0e4a011b4..1ec86d0b0b77 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java @@ -77,7 +77,8 @@ public void setup() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); this.overlordClient = new NoopOverlordClient() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java index 65eeee9af51d..75f4f8a2357e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java @@ -81,7 +81,8 @@ public void setup() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final OverlordClient overlordClient = new NoopOverlordClient(); intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java index c318e85c137d..8d54885610c1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java @@ -111,7 +111,8 @@ public void setup() throws IOException false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final OverlordClient overlordClient = new NoopOverlordClient(); if (LOCAL.equals(intermediateDataStore)) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java index af8bef4e8758..9dd2facbc762 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java @@ -103,7 +103,8 @@ public Period getIntermediaryPartitionTimeout() false, false, TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null + null, + false ); final OverlordClient overlordClient = new NoopOverlordClient() { diff --git a/integration-tests/k8s/role-and-binding.yaml b/integration-tests/k8s/role-and-binding.yaml index ef15b6b74ee1..0de1f196c656 100644 --- a/integration-tests/k8s/role-and-binding.yaml +++ b/integration-tests/k8s/role-and-binding.yaml @@ -19,10 +19,11 @@ metadata: name: druid-cluster rules: - apiGroups: - - "" + - '*' resources: - pods - configmaps + - jobs verbs: - '*' --- diff --git a/integration-tests/k8s/tiny-cluster.yaml b/integration-tests/k8s/tiny-cluster.yaml index cfdacbea23f7..32405e9a5267 100644 --- a/integration-tests/k8s/tiny-cluster.yaml +++ b/integration-tests/k8s/tiny-cluster.yaml @@ -90,7 +90,7 @@ spec: # # Extensions # - druid.extensions.loadList=["druid-avro-extensions","druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-kubernetes-extensions"] + druid.extensions.loadList=["druid-avro-extensions","druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-kubernetes-extensions", "druid-kubernetes-overlord-extensions"] # # Service discovery @@ -160,7 +160,7 @@ spec: volumes: - name: data-volume hostPath: - path: REPLACE_VOLUMES/tmp + path: ${REPLACE_VOLUMES}/tmp resources: requests: memory: "800Mi" @@ -201,6 +201,16 @@ spec: druid.coordinator.asOverlord.enabled=true druid.coordinator.asOverlord.overlordService=druid/overlord druid.indexer.queue.startDelay=PT30S + + # k8s options + druid.indexer.runner.type=${TASK_MODE} + druid.indexer.runner.namespace=${NAMESPACE} + druid.indexer.task.enableTaskLevelLogPush=true + druid.indexer.runner.javaOptsArray=["-server -Xms128m -Xmx128m -XX:MaxDirectMemorySize=256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=/druid/data/tmp -XX:+ExitOnOutOfMemoryError -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"] + druid.indexer.task.baseTaskDir=/druid/data/baseTaskDir + druid.indexer.fork.property.druid.processing.buffer.sizeBytes=25000000 + druid.indexer.fork.property.druid.processing.numMergeBuffers=2 + druid.indexer.fork.property.druid.processing.numThreads=1 extra.jvm.options: |- -Xmx800m -Xms800m @@ -210,7 +220,7 @@ spec: volumes: - name: data-volume hostPath: - path: REPLACE_VOLUMES/tmp + path: ${REPLACE_VOLUMES}/tmp resources: requests: memory: "1G" @@ -254,7 +264,7 @@ spec: volumes: - name: data-volume hostPath: - path: REPLACE_VOLUMES/tmp + path: ${REPLACE_VOLUMES}/tmp resources: requests: memory: "1G" @@ -338,7 +348,7 @@ spec: volumes: - name: data-volume hostPath: - path: REPLACE_VOLUMES/tmp + path: ${REPLACE_VOLUMES} /tmp resources: requests: memory: "1G" diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index bdcea9605f09..dff20ee2ccc8 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -693,6 +693,7 @@ int-test default false + k8s @@ -716,6 +717,7 @@ ${pod.name} ${pod.namespace} ${build.druid.cluster} + ${druid.indexer.mode} ${project.basedir}/script/build_run_k8s_cluster.sh diff --git a/integration-tests/script/setup_druid_on_k8s.sh b/integration-tests/script/setup_druid_on_k8s.sh index 34010f23142c..471c389cc6ce 100755 --- a/integration-tests/script/setup_druid_on_k8s.sh +++ b/integration-tests/script/setup_druid_on_k8s.sh @@ -41,8 +41,10 @@ mkdir tmp chmod 777 tmp $KUBECTL apply -f integration-tests/k8s/role-and-binding.yaml -sed -i "s|REPLACE_VOLUMES|`pwd`|g" integration-tests/k8s/tiny-cluster.yaml -$KUBECTL apply -f integration-tests/k8s/tiny-cluster.yaml +export REPLACE_VOLUMES="$(pwd)" +export TASK_MODE=${INDEXER_MODE:-k8s} +export NAMESPACE=${NAMESPACE:-default} +envsubst < integration-tests/k8s/tiny-cluster.yaml | $KUBECTL apply -f - # Wait a bit sleep 180 diff --git a/integration-tests/script/setup_druid_operator_on_k8s.sh b/integration-tests/script/setup_druid_operator_on_k8s.sh index d99ec6a2bf65..df91e8c26d50 100755 --- a/integration-tests/script/setup_druid_operator_on_k8s.sh +++ b/integration-tests/script/setup_druid_operator_on_k8s.sh @@ -26,13 +26,13 @@ git clone https://github.com/druid-io/druid-operator.git cd druid-operator git checkout -b druid-operator-$DRUID_OPERATOR_VERSION druid-operator-$DRUID_OPERATOR_VERSION cd .. -sed -i "s|REPLACE_IMAGE|druidio/druid-operator:$DRUID_OPERATOR_VERSION|g" druid-operator/deploy/operator.yaml +export REPLACE_ME="druidio/druid-operator:$DRUID_OPERATOR_VERSION" # Deploy Druid Operator and Druid CR spec -$KUBECTL create -f druid-operator/deploy/service_account.yaml -$KUBECTL create -f druid-operator/deploy/role.yaml -$KUBECTL create -f druid-operator/deploy/role_binding.yaml -$KUBECTL create -f druid-operator/deploy/crds/druid.apache.org_druids_crd.yaml -$KUBECTL create -f druid-operator/deploy/operator.yaml +$KUBECTL apply -f druid-operator/deploy/service_account.yaml +$KUBECTL apply -f druid-operator/deploy/role.yaml +$KUBECTL apply -f druid-operator/deploy/role_binding.yaml +$KUBECTL apply -f druid-operator/deploy/crds/druid.apache.org_druids_crd.yaml +envsubst < druid-operator/deploy/operator.yaml | $KUBECTL apply -f - echo "Setup Druid Operator on K8S Done!" diff --git a/licenses.yaml b/licenses.yaml index 6d7e732310c8..6208638ed608 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -897,6 +897,16 @@ libraries: --- +name: kubernetes fabric java client +license_category: binary +module: extensions-contrib/kubernetes-overlord-extensions +license_name: Apache License version 2.0 +version: 5.12.2 +libraries: + - io.fabric8: kubernetes-client + +--- + name: io.prometheus simpleclient_common license_category: binary module: extensions/druid-kubernetes-extensions diff --git a/pom.xml b/pom.xml index 316661fe6537..c0032215a15c 100644 --- a/pom.xml +++ b/pom.xml @@ -212,6 +212,7 @@ extensions-contrib/aliyun-oss-extensions extensions-contrib/prometheus-emitter extensions-contrib/opentelemetry-emitter + extensions-contrib/kubernetes-overlord-extensions distribution diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java index 58dd8ae24215..821e4343901f 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java @@ -209,6 +209,7 @@ public int getTotalWorkerCapacity() response.getStatus(), response.getContent() ); + } final Collection workers = jsonMapper.readValue( response.getContent(), diff --git a/server/src/main/java/org/apache/druid/guice/annotations/AttemptId.java b/server/src/main/java/org/apache/druid/guice/annotations/AttemptId.java new file mode 100644 index 000000000000..6972942b780f --- /dev/null +++ b/server/src/main/java/org/apache/druid/guice/annotations/AttemptId.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.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface AttemptId +{ +} diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 3a45975bf9d7..7aee80ccbd25 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -95,6 +95,9 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorModule; import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.indexing.worker.shuffle.DeepStorageIntermediaryDataManager; +import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; +import org.apache.druid.indexing.worker.shuffle.LocalIntermediaryDataManager; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; @@ -233,6 +236,7 @@ public void configure(Binder binder) binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); configureTaskStorage(binder); + configureIntermediaryData(binder); configureAutoscale(binder); configureRunners(binder); configureOverlordHelpers(binder); @@ -291,6 +295,21 @@ private void configureTaskStorage(Binder binder) storageBinder.addBinding("metadata").to(MetadataTaskStorage.class).in(ManageLifecycle.class); binder.bind(MetadataTaskStorage.class).in(LazySingleton.class); } + private void configureIntermediaryData(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.processing.intermediaryData.storage.type", + Key.get(IntermediaryDataManager.class), + Key.get(LocalIntermediaryDataManager.class) + ); + final MapBinder biddy = PolyBind.optionBinder( + binder, + Key.get(IntermediaryDataManager.class) + ); + biddy.addBinding("local").to(LocalIntermediaryDataManager.class); + biddy.addBinding("deepstore").to(DeepStorageIntermediaryDataManager.class).in(LazySingleton.class); + } private void configureRunners(Binder binder) { diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index f46eb4141528..cc9a4cf9c451 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -46,6 +46,7 @@ import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; +import org.apache.druid.guice.IndexingServiceTaskLogsModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.Jerseys; import org.apache.druid.guice.JoinableFactoryModule; @@ -58,6 +59,7 @@ import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.guice.annotations.AttemptId; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; @@ -123,8 +125,8 @@ import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.eclipse.jetty.server.Server; -import java.io.File; import java.io.IOException; +import java.nio.file.Paths; import java.util.List; import java.util.Properties; import java.util.Set; @@ -140,18 +142,15 @@ public class CliPeon extends GuiceRunnable { @SuppressWarnings("WeakerAccess") - @Arguments(description = "task.json status.json report.json") @Required + @Arguments(description = "taskDir attemptId") public List taskAndStatusFile; - // path to store the task's stdout log - private String taskLogPath; + // path to the task Directory + private String taskDirPath; - // path to store the task's TaskStatus - private String taskStatusPath; - - // path to store the task's TaskReport objects - private String taskReportPath; + // the attemptId + private String attemptId; /** * Still using --nodeType as the flag for backward compatibility, although the concept is now more precisely called @@ -193,20 +192,21 @@ protected List getModules() new QueryableModule(), new QueryRunnerFactoryModule(), new JoinableFactoryModule(), + new IndexingServiceTaskLogsModule(), new Module() { @SuppressForbidden(reason = "System#out, System#err") @Override public void configure(Binder binder) { - taskLogPath = taskAndStatusFile.get(0); - taskStatusPath = taskAndStatusFile.get(1); - taskReportPath = taskAndStatusFile.get(2); + taskDirPath = taskAndStatusFile.get(0); + attemptId = taskAndStatusFile.get(1); binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/peon"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); binder.bind(ResponseContextConfig.class).toInstance(ResponseContextConfig.newConfig(true)); + binder.bindConstant().annotatedWith(AttemptId.class).to(attemptId); JsonConfigProvider.bind(binder, "druid.task.executor", DruidNode.class, Parent.class); @@ -218,14 +218,22 @@ public void configure(Binder binder) binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); LifecycleModule.register(binder, ExecutorLifecycle.class); - binder.bind(ExecutorLifecycleConfig.class).toInstance( - new ExecutorLifecycleConfig() - .setTaskFile(new File(taskLogPath)) - .setStatusFile(new File(taskStatusPath)) - ); + ExecutorLifecycleConfig executorLifecycleConfig = new ExecutorLifecycleConfig() + .setTaskFile(Paths.get(taskDirPath, "task.json").toFile()) + .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()); + + if ("k8s".equals(properties.getProperty("druid.indexer.runner.type", null))) { + log.info("Running peon in k8s mode"); + executorLifecycleConfig.setParentStreamDefined(false); + } + + binder.bind(ExecutorLifecycleConfig.class).toInstance(executorLifecycleConfig); binder.bind(TaskReportFileWriter.class) - .toInstance(new SingleFileTaskReportFileWriter(new File(taskReportPath))); + .toInstance( + new SingleFileTaskReportFileWriter( + Paths.get(taskDirPath, "attempt", attemptId, "report.json").toFile() + )); binder.bind(TaskRunner.class).to(SingleTaskBackgroundRunner.class); binder.bind(QuerySegmentWalker.class).to(SingleTaskBackgroundRunner.class); diff --git a/services/src/test/java/org/apache/druid/cli/MainTest.java b/services/src/test/java/org/apache/druid/cli/MainTest.java index 470a763a1387..e90f2745ff98 100644 --- a/services/src/test/java/org/apache/druid/cli/MainTest.java +++ b/services/src/test/java/org/apache/druid/cli/MainTest.java @@ -27,11 +27,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; import java.util.Properties; -/** - * This test verifies injection for {@link ServerRunnable}s which are discoverable Druid servers. - */ @RunWith(Parameterized.class) public class MainTest { @@ -41,6 +41,8 @@ public static Iterable constructorFeeder() return ImmutableList.of( new Object[]{new CliOverlord()}, new Object[]{new CliBroker()}, + // Takes arguments. Cannot be used in this test + new Object[]{new FakeCliPeon(true)}, new Object[]{new CliHistorical()}, new Object[]{new CliCoordinator()}, new Object[]{new CliMiddleManager()}, @@ -63,4 +65,31 @@ public void testSimpleInjection() injector.injectMembers(runnable); Assert.assertNotNull(runnable.makeInjector(runnable.getNodeRoles(new Properties()))); } + + private static class FakeCliPeon extends CliPeon + { + List forkTaskAndStatusFile = new ArrayList(); + + FakeCliPeon(boolean runningOnK8s) + { + forkTaskAndStatusFile.add("src/test/resources"); + forkTaskAndStatusFile.add("task_id"); + forkTaskAndStatusFile.add("1"); + + try { + Field privateField = CliPeon.class + .getDeclaredField("taskAndStatusFile"); + privateField.setAccessible(true); + privateField.set(this, forkTaskAndStatusFile); + + if (runningOnK8s) { + System.setProperty("druid.indexer.runner.type", "k8s"); + } + } + catch (Exception ex) { + // do nothing. + } + + } + } }