You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/09/29 21:46:46 UTC

[GitHub] [druid] churromorales opened a new pull request, #13156: Support for middle manager less druid, tasks launch as k8s jobs

churromorales opened a new pull request, #13156:
URL: https://github.com/apache/druid/pull/13156

   
   ### Description
   
   Add an extension to allow tasks to be run as k8s jobs from the overlord, eliminating the need for a middle manager.
   
   The core changes are as follows: 
   1. Refactored arguments to CliPeon to be more generic
   2. Had to add a setup and cleanup method to AbstractTask.  
   	a. Because tasks run on separate pods, the task needs to setup its own filesystem directories. 
   	b. Again because the tasks run on separate pods, we push the task logs from the task itself and task reports. in the cleanup method. 
   3. A few other small changes to core required for tasks to run independently on their own.
   
   ### How it works
   
   The KubernetesTaskRunner runs in the overlord process.  When it has a request to launch a task, it goes to the K8sApi, grabs its own PodSpec (the overlord itself).  Takes that podSpec, modifies the necessary attributes (eg: command, labels, env variables etc).  Takes the task.json, compresses and base64 encodes it.  Then launches a K8s Job.  
   
   The K8s Job on startup, will unwrap the task.json env variable, write it to the appropriate directory and run the task. 
   
   The KubernetesTaskRunner monitors the lifecycle of the task, just as the ForkingTaskRunner and returns the TaskStatus.
   
   #### What if you are running Sidecars?
   The config option `druid.indexer.runner.sidecarSupport` will support launching sidecars, I utilize kubexit (https://github.com/karlkfi/kubexit) to setup the spec such that when the main container completes, it terminates the sidecars.  This is a known issue with k8s jobs and this is how I work around it.
   
   #### Another nice side-effect
   Because the launching of tasks has been decoupled from the service itself, the tasks run independently regardless of the state of the overlord process.  You can shut down the overlord process, and when it comes back.  It will go to the k8s api and get the status of all peon jobs regardless of phase (in flight, completed, failed, pending) and will do the proper bookeeping for completed tasks and will resume monitoring tasks in flight.
   
   To run a middle manager less druid, simply omit the middle manager from your deployment. 
   
    Make sure you also change `druid.processing_intermediaryData.storage.type=deepStorage`
   
   In your overlord config: 
   	1. Add the `druid-kubernetes-overlord-extensions` to your extensions load list.
   	2. `druid.indexer.runner.type=k8s`
   	3. `druid.indexer.runner.namespace=<currentNamespace>`
   	4. `druid.indexer.task.enableTaskLevelLogPush=true` (optional but recommended.
   
   This PR has:
   - [ x] been self-reviewed.
      - [x ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ x] added documentation for new or modified features or behaviors.
   - [ x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ x] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests. (this has been added but the k8s integration tests only work on a linux machine as they use `conntrack` with minikube.  Thus I will have to let travis run and figure things out from there.
   - [x] been tested in a test Druid cluster.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1264020047

   This pull request **introduces 5 alerts** when merging 2f06ec11211608548546c28b3b4d0384f5d5b018 into ebfe1c0c90d86e4d188617fe840dafb2c9b7e5b0 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-0ff9347b65109a572974a6e958ee9bf5c7948440)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007151221


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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|
+

Review Comment:
   I can definitely put this in this PR.  Would it be okay to add in the PR comments or do you want this in the code?  So I did add a sample spec to the integration tests, so you can see an example operator yaml on how to deploy with this feature turned on: `tiny-cluster.yaml`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007162501


##########
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<String, TaskRunnerFactory> 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<String, TaskLogs> 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<? extends Module> getJacksonModules()
+  {
+    return Collections.emptyList();
+  }

Review Comment:
   thanks for the suggestion, I will do that. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008535210


##########
indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java:
##########
@@ -927,6 +932,23 @@ public Response apply(TaskRunner taskRunner)
           {
             if (taskRunner instanceof WorkerTaskRunner) {
               return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build();
+            } else if (taskRunner.isK8sTaskRunner()) {

Review Comment:
   of courser. 



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -125,6 +139,69 @@ protected AbstractTask(
     this(id, groupId, taskResource, dataSource, context, IngestionMode.NONE);
   }
 
+  @Nullable
+  public String setup(TaskToolbox toolbox) throws Exception

Review Comment:
   yes sir!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005886988


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.

Review Comment:
   Additionally anything that communicates with the peon pod has to be in the same namespace. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1301709751

   Thank you for the contribution @churromorales. This will be a major feature in the upcoming 25.0.0 release. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995007769


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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));

Review Comment:
   this is for the peon task.  For the MM based peon tasks, these were dynamic because you were running multiple peons on a host. So the host was static, and you had to make the ports dynamic. 
   
   Now in the k8s world, each job pod will have a unique host, so we can use a static port (we don't need it dynamic to avoid collisions). 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1278189632

   This pull request **introduces 5 alerts** when merging 6c13bf0a72fab4d6f66a502ba35920afcc0d6fbb into 45dfd679e92e668172c470c37615e7447b601af1 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-46b961765f5a8a0fe9806e1eb60566a65ba05756)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995000617


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   So the reason for the podSpec is that I didn't want to take a brand new docker image and create a job spec for it.  I don't know your cluster config.  Suppose you have secrets mounted, volumes setup up your way, env variables, certs, annotations for things like istio, etc... The parent pod spec logic is to  ensure you have those things for your peon task and you don't have to worry about it.  But, I do take that pod spec and massage it.  I do things like change the resources required.  For CPU we always give the task a single core (just like what druid was always doing).  For memory we take the jvm opts you pass and configure the container resources from that.  I believe its something like (Xmx + dbb)*1.2.  So while the cpu resources are fixed at a core, the memory is deduced from your jvm opts.  I want to make sure I answered your question correctly here, please let me know things don't make sense. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1287495014

   This pull request **introduces 5 alerts** when merging 27d29fb77b1bf8203b1a91a8c05e52f240948a8d into 86e6e61e884230e7a82e726f5f5c33602b242caf - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-53a102b209a9668303f97f3087d3dfd9ba3a6a11)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1286251804

   This pull request **introduces 5 alerts** when merging e491abc2bee159adeff1d2b1d79b03f21b621b54 into fc262dfbaf434614a2ef550775b0c6b736856e28 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-eec40d8b51b8bf6b421fa72db60b0c599f118ce2)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales closed pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales closed pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs
URL: https://github.com/apache/druid/pull/13156


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1297750038

   >     * Avoid creating TaskLocations with `null` host and non-`-1` values for `port` and `tlsPort`. This may be happening in `K8sWorkItem` if `mainPod.getStatus().getPodIP()` is `null`. Do you think this is possible? To keep things coherent I suggest enforcing it in the TaskLocation constructor, which could throw an exception if host is `null` and `port` and `tlsPort` are anything other than `-1`. (This would ensure that `null` host always compares equal to `TaskLocation.unknown()`.)
   
   @gianm thats absolutely the problem I believe, thank you for pointing me in the right direction, I couldn't tell from the trace where the issue was.  I'll fix that in a follow up PR so we can get these two features working together.  Thank you again for getting back to me about this. 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1291145564

   @gianm I was testing the MM-less patch on the msq work you did.  I ran a test ingestion and the tasks just hang forever, after a bit of debugging here is what is happening, launch a controller with one worker: 
   
   I get this exception: 
   
   ```
   2022-10-25T20:32:25,413 ERROR [ServiceClientFactory-0] com.google.common.util.concurrent.ExecutionList - RuntimeException while executing runnable com.google.common.util.concurrent.Futures$4@7f14c4aa with executor com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService@138e191f
   java.lang.NullPointerException: host
   	at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:229) ~[guava-16.0.1.jar:?]
   	at org.apache.druid.rpc.ServiceLocation.<init>(ServiceLocation.java:39) ~[druid-server-24.0.0-6.jar:24.0.0-6]
   	at org.apache.druid.rpc.indexing.SpecificTaskServiceLocator$1.onSuccess(SpecificTaskServiceLocator.java:137) ~[druid-server-24.0.0-6.jar:24.0.0-6]
   	at org.apache.druid.rpc.indexing.SpecificTaskServiceLocator$1.onSuccess(SpecificTaskServiceLocator.java:113) ~[druid-server-24.0.0-6.jar:24.0.0-6]
   	at com.google.common.util.concurrent.Futures$4.run(Futures.java:1181) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.AbstractFuture.set(AbstractFuture.java:185) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.Futures$ChainingListenableFuture$1.run(Futures.java:872) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.Futures$ImmediateFuture.addListener(Futures.java:102) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.Futures$ChainingListenableFuture.run(Futures.java:868) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.AbstractFuture.set(AbstractFuture.java:185) ~[guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.SettableFuture.set(SettableFuture.java:53) ~[guava-16.0.1.jar:?]
   	at org.apache.druid.rpc.ServiceClientImpl$1.onSuccess(ServiceClientImpl.java:194) ~[druid-server-24.0.0-6.jar:24.0.0-6]
   	at org.apache.druid.rpc.ServiceClientImpl$1.onSuccess(ServiceClientImpl.java:168) ~[druid-server-24.0.0-6.jar:24.0.0-6]
   	at com.google.common.util.concurrent.Futures$4.run(Futures.java:1181) ~[guava-16.0.1.jar:?]
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
   	at java.lang.Thread.run(Thread.java:829) ~[?:?]
   ```
   
   For the middle manager less patch in k8s.  I launch a task, I added a setup and teardown functions, so before every AbstractTask runs, it will announce it's own location, then on teardown update status etc...
   
   From what I see here, is that we do have a taskStatus (the task gets launched) but the location has not yet been announced, in k8s we don't know the location until the pod comes up and the service is available to take a request.  So in the msq patch, it doesn't wait for the location, it assumes it knows it.  But we need this for the MM-less patch.  
   
   TLDR, its a race, we try to get the location for the controller before it announces it.  The TaskLocation is `unknown` until the task's runTask() method is invoked.  The precondition on a null host in the ServiceLocation constructor causes everything to die. 
   
   Do you have any advice how we can make these two co-exist?  This is the only blocker I see for this to work, everything else works as it did before.  I can't figure out a clean way, also I don't fully understand the msq patch, I thought you might have a solution for this. 
   
   Thank you
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1295436285

   This pull request **introduces 5 alerts** when merging b12de57c02930116aa72fccb86e5e8f7aa94e373 into 4f0145fb8506257f6531cce04b5dee8354a1a1a1 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-ffe5e83e66bba22e71cd4739fe2833a7612d6595)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007160897


##########
extensions-contrib/kubernetes-overlord-extensions/pom.xml:
##########
@@ -0,0 +1,233 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.druid.extensions.contrib</groupId>
+  <artifactId>druid-kubernetes-overlord-extensions</artifactId>
+  <name>druid-kubernetes-overlord-extensions</name>
+  <description>druid-kubernetes-overlord-extensions</description>
+
+  <parent>
+    <groupId>org.apache.druid</groupId>
+    <artifactId>druid</artifactId>
+    <version>25.0.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-server</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.squareup.okhttp3</groupId>
+          <artifactId>okhttp</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.squareup.okhttp3</groupId>
+          <artifactId>logging-interceptor</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-processing</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-indexing-service</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>2.6</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>2.11.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>3.12.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-common</artifactId>
+      <version>4.1.68.Final</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <version>1.26.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-text</artifactId>
+      <version>1.9</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-model-core</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+    <dependency>
+      <groupId>javax.validation</groupId>
+      <artifactId>validation-api</artifactId>
+      <version>1.1.0.Final</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-model-batch</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-client</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+
+    <!-- Tests -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-server-mock</artifactId>
+      <version>5.12.2</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-api</artifactId>
+      <version>5.8.2</version>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- others -->
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject.extensions</groupId>
+      <artifactId>guice-multibindings</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.9.5</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-indexing-service</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-server</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-processing</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+</project>

Review Comment:
   ill fix that 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] a2l007 merged pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
a2l007 merged PR #13156:
URL: https://github.com/apache/druid/pull/13156


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005880461


##########
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 ;;

Review Comment:
   i can combine it in the druid.sh if you want...I'm totally happy to do that. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r984139254


##########
distribution/docker/peon.sh:
##########
@@ -0,0 +1,153 @@
+#!/bin/sh

Review Comment:
   Meta commentary: it would be cool (and I don't think too crazy) if there was a Cli target that is like "task" which essentially does a remote bootstrap.
   
   I.e. it could start up and the first thing it does is read some config for the overlord process, "phone home" to ask for the task spec and runtime properties, and then use those to bootstrap.  
   
   This might open up security holes, so it would probably have to be done with some sort of shared secret or something, maybe?  But anyway, might simplify this script into something that is relatively generic and not even k8s-dependent.
   
   Could also be done as an evolution of this change post-merge.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] Fryuni commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
Fryuni commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1278322121

   Hey, this is an awesome PR, thanks for all the work
   I am concerned about this part, though:
   
   > The KubernetesTaskRunner runs in the overlord process. When it has a request to launch a task, it goes to the K8sApi, grabs its own PodSpec (the overlord itself). Takes that podSpec, modifies the necessary attributes (eg: command, labels, env variables etc). Takes the task.json, compresses and base64 encodes it. Then launches a K8s Job.
   
   The resource use for overlords and tasks is vastly different, using the same podSpec for those would require reserving much more resources for the overlords and the tasks than necessary.
   
   One idea is that instead of reading from the overlord pod itself, it would be better to provide the name of a config map containing the podSpec.
   
   Another idea: the middle-managers have tiers, which we can use to allocate tasks to middle-managers of different tiers. Those tiers could be used to define different podSpecs for different tasks without changing the tasks API.
   
   Those changes can be done on a later PR. All in all, this PR is a huge step forward in making Druid more scalable for cloud environments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] georgew5656 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
georgew5656 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r994793849


##########
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<String, String> annotations = addJobSpecificAnnotations(context, k8sTaskId);
+    Map<String, String> 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<String> originalCommand = container.getCommand();
+    List<String> 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<Container> 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) {

Review Comment:
   is there any harm in just letting this run for a pod with no sidecars? it looks like we'll just add the kubexit init container and a couple env variables but not any KUBEXIT_DEATH_DEPS (so we ought to just exit cleanly)
   
   then the logic can all be moved into K8sTaskAdapter



##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   similar to the other comments, I think we want to make additional things configurable here if possible



##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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<String> mainCommand = Lists.newArrayList("sh", "-c");
+    // update the command
+    List<Container> 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<String, Quantity> resources = ImmutableMap.of(
+        "cpu",
+        new Quantity("1000", "m"),

Review Comment:
   i think this should be a configuration in KubernetesTaskRunnerConfig as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] georgew5656 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
georgew5656 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r996063207


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   i also like the idea of using a custom pod spec, but that seems like a bigger change that we could do as a followup



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008535341


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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 java.io.InputStream;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public interface KubernetesPeonClient

Review Comment:
   you got it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007169332


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());

Review Comment:
   yes, good catch!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007167423


##########
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.

Review Comment:
   so the way this works is this. 
   
   Overlord is up, launches a task....then you shut down druid (everything suppose).  Overlord starts up and there is a call to getKnownTasks() which in this implementation calls out to k8s to get all peon k8s jobs (in any state). 
   
   It then calls run(), run checks for task states and does the right thing, running tasks will show you logs and completed tasks will have bookkeeping done.  So tasks run independently (mostly) of the cluster itself (still have to work about getting locks from the overlord) but they are not tied to the overlord process at all like they were the middle manager process. 
   
   Does this make sense?  I guess what I am trying to say is if you look at the run() method we do checks for completed tasks, running tasks, and tasks that have not been launched.  So what we are doing is when the overlord starts up, it just gets the knownTasks and calls run() and run() will do the right thing. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1297821985

   This pull request **introduces 5 alerts** when merging fb6609e189e8f041219821ef349e401b6ba28cdb into 675fd982fb5ca274b057495a90563ecc248ad823 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-e5b79e551626bde9fdcccc3e7e4e04cfbf508e37)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1262924253

   This pull request **introduces 5 alerts** when merging 190f8dd0c0132ee55c4e9581efac4a305403f37f into ce5f55e5ce00d876277424ea0724b70a330315b3 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-25607c603af6b5b6dcd227e784a6c429a4deebe5)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995014835


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   agreed, i will make env's and annotations configurable. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995013773


##########
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<String, String> annotations = addJobSpecificAnnotations(context, k8sTaskId);
+    Map<String, String> 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<String> originalCommand = container.getCommand();
+    List<String> 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<Container> 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) {

Review Comment:
   So technically there is no `harm`.  But consider the situation where a user can't pull from dockerhub, they also have to have some cicd pipeline to push docker images for security reasons.  Now this user runs druid without any sidecars, so now they have to jump through unnecessary hoops to use this patch.  If you want to run stock without sidecars, I feel the user should not have to do extra work.  If you want to run with sidecars, then this is the best way I could come up with to solve the problem.  You are totally correct it will work with a single container, but the user will have to add dependencies for no reason, I was trying to avoid that. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995049233


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   > Perhaps provide just a bit more background for newbies. Where does the pod spec come from? Something like:
   > 
   > > With this extension, your Overload tasks run in a Kubernetes pod. You define that pod using a pod spec. The pod spec names a Docker image with Druid installed, identifies K8s settings, provides Druid configuration, passes in secrets, etc. You create the pod spec outside of Druid. We suggest you test the spec directly in K8s before using it with Druid.
   > 
   > I'm sure some (all?) of the above is wrong: it just identifies what I, as a newbie to this extension, would need to know. Focus in particular on anything Druid-specific I need to provide in the pod spec.
   > 
   > Also, where do I put the pod spec? In K8s somewhere? On the local disk of each Druid node?
   
   Maybe I wasn't clear, you don't have to do anything here.  The podSpec comes from the overlord pod, the pod that launches the task.  The `K8sTaskAdapter` takes the overlord spec and modifies it to run a peon k8s job.  The user has to do nothing here.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] georgew5656 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
georgew5656 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r996060393


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   I thought I added it as a comment but what I actually think is most important to be configurable is nodeSelector/affinity/tolerations in case people want to configure what node pools to run on.
   
   If nodeSelector is not set than maybe we can use whatever the overlord uses as its node selector.
   
   Env variables is not as important, but I could see something like DD_TRACE_AGENT_URL for enabling tracing with datadog



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005879892


##########
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"

Review Comment:
   this is a good question, the service needs to be overlord, and the command is different.  I also didn't want to modify the druid.sh script because this is an extension.  I want to keep things totally different here. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007160434


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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"}'`

Review Comment:
   so i thought about that, I would have to duplicate the IndexingServiceModuleHelper just for this extension.  How about I make you a deal, we can do all the cleanup in a following PR.  Honestly there are a few hacks because this is an extension, if it gets into core, I can fix up a lot of the code.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1298942861

   This pull request **introduces 5 alerts** when merging a4b4cdb1b021a097627963b74cccbf7adb431d91 into fd7864ae33ba8afe56d660c2b6d84ec5b498d321 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-7b2e981976c5817eb3dfc150bf0892322208f9a7)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1299279213

   This pull request **introduces 5 alerts** when merging ab612d83fb190f98a106e4ca70059f95f470d986 into 176934e849d50a2360f50db7771c1775110308f3 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-9c59d898bf344d10cd0f6c193701f2fffd3e207c)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] a2l007 commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
a2l007 commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1301581547

   Thanks for the contribution @churromorales !
   Looking forward to the followup PRs to handle the outstanding items identified from this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005527356


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.  

Review Comment:
   I think it's the role that is attached to the overlord pod. So maybe through the overlord pod spec



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1294215176

   This pull request **introduces 5 alerts** when merging a23bba92d722562a7342b8283978a187679a7a1f into affc522b9f8227a42631e55bc9ad833ee57fc3a0 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-5229d0199f1c704f40be8dddae36ccc9bede220e)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] Fryuni commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
Fryuni commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1292666488

   @abhishekagarwal87 
   > may not be an issue but is there a limit in k8s on how large the env variable value can be? The task JSON could be large.
   
   I just checked the defaults for K8s, k3s, microk8s, GKE, and EKS:
   - A configMap value is limited to 1MiB
   - Similarly, an inline `env` value is limited to 1MiB
   - The entire final command passed to the container (command + arguments + ENVs) is limited to the `ARG_MAX` configuration of the host node, which defaults to 2MiB on Linux.
   
   None of our tasks reach close to a mebibyte, but if that could be a problem, then the best would be retrieving the task from the coordinator.
   
   Mounting a config map instead of passing as an env var is cleaner IMHO, but would only make a difference regarding limits if there were multiple envs smaller than 1MiB that combined go over 2MiB.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007223170


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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);

Review Comment:
   I believe task status is only success or failure.  The setup() and cleanup() methods of Abstract task do the work of reporting back.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008145988


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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"}'`

Review Comment:
   I understand. I just checked and other runner types are not namespacing their configuration as well. So I think it's ok. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1281201211

   > > This doesn't use the resources of the overlord, we overwrite the resources for podSpec. The memory is derived from your JAVA_OPTS (for your task) basically 1.2*(Xmx + Dbb) for the memory limit and the cpu is always 1.
   > 
   > That is super great news on the memory side, but not so great on the CPU side for us. We have 23 continuously running ingestion tasks and they only use a tenth of a core (100m) each, so fixing the resource request to one core is a 9x overprovision totaling 21 cores. A full core works for our compaction tasks, but those run for about 10 minutes every 4 hours.
   > 
   > We'll explore some things with Druid in the near future, so we might use this PR before it lands and give some feedback.
   
   I agree, we need some extra configuration, I think that can come in a subsequent PR as the more folks that use this, the more we will find that needs to be configurable.  I like the idea of having the current model be the default and then for users that want something more customized, perhaps we can have a configmap based podSpec they could load as a base template.  I think that could please most folks, while still providing an easy configuration for those users that want to just to have jobs run in k8s.  
   
   I just pushed up a commit, I had it run on a few clusters for 4-5 days and everything seems good.  If you want to test this patch, I would recommend using the latest commit from this branch as it reduces the time spent locking dramatically.  Let me know if you guys have any other concerns about this patch.  If you have already done a review, you can look at the last commit and see what changed.  Thanks again for the reviews. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] georgew5656 commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
georgew5656 commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1270420351

   this looks great @churromorales, i've started looking through some of the kubernetes related code today, will hopefully finish up by tomorrow


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995011070


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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");

Review Comment:
   Good question.  For this patch, I am forcing the tasks to run in the current namespace that druid is running in.  You could easily have the tasks run in a different namespace but would require a bit of changes to this patch (not much though).  So when a job gets launched, we wait for the pod to startup, once the pod starts, we grab the ip address of the pod and broadcast that as the `TaskLocation`.  There would be no need for a proxy if you wanted to go across namespaces in the same context, but if you had druid running in namespace `A` and tasks running in namespace `B` you would have to setup your k8s to be able to communicate bidirectionally.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995056915


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   > I'm sure some (all?) of the above is wrong: it just identifies what I, as a newbie to this extension, would need to know. Focus in particular on anything Druid-specific I need to provide in the pod spec.
   > 
   > Also, where do I put the pod spec? In K8s somewhere? On the local disk of each Druid node?
   
   Should i just exclude the podSpec part?  Users have to do nothing here, I am just explaining how it works at a high-level.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1301397997

   This pull request **introduces 5 alerts** when merging d40db8f8226b4d0da7325d55d7ad8f3367de64a4 into 176934e849d50a2360f50db7771c1775110308f3 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-90496df876082e586cdd53722a183a41393a34eb)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995022083


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   actually one quesiton, for env variables being configurable, do you have a usecase?  It takes all the env variables of the overlord pod running, would there be a situation where you would need something additional here?  The container shouldn't need more than what druid has to work, right?  Annotations I will add, because that could be something you have setup for k8s, like some autoscaling annotation you want to add or networking....labels are already configurable.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995068148


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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");

Review Comment:
   The goal of this patch, is if you have are running druid on `k8s`, you shouldn't need to do anything except add the extension and a few configuration options to the overlord and you are good to go.  In the gotchas section of the readme, there are some things you have to look out for with sidecars which i mentioned.   Also your service account must have the ability to deal with k8s jobs, I included a sample role / rolebinding as an example in the readme.md.  We have been using this on some of our clusters for a while now and we haven't really had to add any extra configuration or do anything special in k8s to get this to work.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995063507


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   annotations and labels are configurable, if you can provide me with an example of how we would need env variables, i would be happy to add it as configurable.  But I personally think that could cause more harm than good potentially. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995052737


##########
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.

Review Comment:
   the `peon.sh` is copied into the dockerifle, do we need to add a rule here?  I just do it in the dockerfile itself, user wont have to worry, if anything changes with the script it gets automatically updated.  I believe that is the same thing that happens with the `druid.sh` script. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1294034938

   This pull request **introduces 5 alerts** when merging 5a0c734d9972c7f9319a53044ec1a43d29d88871 into affc522b9f8227a42631e55bc9ad833ee57fc3a0 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-155ddca569119c56ff0c3b08915146d5a613f120)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
gianm commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1294563901

   > @gianm I was testing the MM-less patch on the msq work you did. I ran a test ingestion and the tasks just hang forever, after a bit of debugging here is what is happening, launch a controller with one worker:
   
   Thanks for the details! Looking into it I think the issue is that the SpecificTaskServiceLocator _is_ able to handle tasks with unknown location, but it does this by a logic branch involving `TaskLocation.unknown().equals(statusPlus.getLocation())`. Probably, this branch doesn't get used because the location has `null` host, yet `port` and `tlsPort` are set. So, it does not `equals` to `TaskLocation.unknown()`, and therefore we end up trying to treat it as valid in the service locator.
   
   Couple ways to fix it:
   
   - Avoid creating TaskLocations with `null` host and non-`-1` values for `port` and `tlsPort`. This may be happening in `K8sWorkItem` if `mainPod.getStatus().getPodIP()` is `null`. Do you think this is possible? To keep things coherent I suggest enforcing it in the TaskLocation constructor, which could throw an exception if host is `null` and `port` and `tlsPort` are anything other than `-1`. (This would ensure that `null` host always compares equal to `TaskLocation.unknown()`.)
   - Or, an alternate approach: in SpecificTaskServiceLocator, use `statusPlus.getLocation().getHost() == null` instead of comparing to `TaskLocation.unknown()`. (With this approach, note that there are a bunch of other places that use equality checks on `TaskLocation.unknown()`. They should be updated too.)
   
   I'm happy if we do one of these fixes as a follow-up to your patch. You don't need to do it as part of this initial PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] a2l007 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
a2l007 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008223696


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.google.common.base.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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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, 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(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } 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(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");
+    cleanupExecutor.shutdownNow();
+    log.info("Stopped...");
+  }
+
+  @Override
+  public Map<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> command = new ArrayList<>();
+    // must use deep storage shuffle now with k8s peon tasks.

Review Comment:
   Do we need this comment here?



##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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)

Review Comment:
   Comments might be useful for this method on how the container memory is estimated.



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.google.common.base.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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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, 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(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } 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(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");

Review Comment:
   Can these log lines be more descriptive?



##########
indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java:
##########
@@ -927,6 +932,23 @@ public Response apply(TaskRunner taskRunner)
           {
             if (taskRunner instanceof WorkerTaskRunner) {
               return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build();
+            } else if (taskRunner.isK8sTaskRunner()) {

Review Comment:
   Can we add a comment here describing that k8staskrunner specific handling was needed since it doesn't have the concept of workers and a dummy worker is being returned here?



##########
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

Review Comment:
   Can we add some javadocs here?



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -125,6 +139,69 @@ protected AbstractTask(
     this(id, groupId, taskResource, dataSource, context, IngestionMode.NONE);
   }
 
+  @Nullable
+  public String setup(TaskToolbox toolbox) throws Exception

Review Comment:
   Can we add tests for this and the `cleanup` method? That should help with the Travis branch coverage failures.



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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 java.io.InputStream;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public interface KubernetesPeonClient

Review Comment:
   Can we add some docs here as well?



##########
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.

Review Comment:
   Can we add a comment here specifying that this is similar to druid.sh and is used exclusively for the kubernetes-overlord-extension?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005523775


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   it makes sense to get the podspec from the overlord. But I suppose that I can override some elements in the spec if I want as a user? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008535459


##########
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

Review Comment:
   of course



##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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)

Review Comment:
   good call, will add it 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r984740714


##########
distribution/docker/peon.sh:
##########
@@ -0,0 +1,153 @@
+#!/bin/sh

Review Comment:
   So when creating this patch, I was hoping to minimize core changes to increase the odds of this patch getting accepted.  In all honesty, I believe if this is a first class feature, we can fix some other hacky stuff that was done.  I definitely think having a Cli command for the task is a good idea, I think this is something that I could put up a PR if /when this gets merged.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r984986352


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,96 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   Nit: space between `Overlord` and "pod".



##########
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<Long> getJobDuration()
+  {
+    Optional<Long> duration = Optional.absent();
+    try {
+      if (job.getStatus() != null) {
+        if (job.getStatus().getStartTime() != null) {
+          if (job.getStatus().getCompletionTime() != null) {

Review Comment:
   Nit: and together the conditions, since the only logic appears where all thee conditions are true.



##########
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

Review Comment:
   Nit: misspelling of "volumees"
   
   Nit: "maps, etc..and" --> "maps, etc. and"



##########
services/src/test/java/org/apache/druid/cli/CliPeonTest.java:
##########
@@ -0,0 +1,73 @@
+package org.apache.druid.cli;

Review Comment:
   Checkstyle will complain due to lack of license header. Suggestion: run checkstyle before each commit, it is fast:
   
   ```bash
   mvn checkstyle:checkstyle -nsu
   ```



##########
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

Review Comment:
   Nit: missing newline, here and below.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1279280242

   Thank you for your kind words.  I hope this patch will be useful to folks upstream.
   
   > The resource use for overlords and tasks is vastly different, using the same podSpec for those would require reserving much more resources for the overlords and the tasks than necessary.
   
   This doesn't use the resources of the overlord, we overwrite the resources for podSpec.  The memory is derived from your JAVA_OPTS basically 1.2*(Xmx + Dbb) and the cpu is always 1.  You get a core per task just like we did before.  We only grab certain items from the parent podSpec.  You can see the logic in the `K8sTaskAdapter`
   
   I am open to whatever changes you guys think are necessary.  I have a few more ideas for subsequent PRs and I do like the configmap pod spec for users that want more control.  That might be a configurable option we can provide in another PR. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1301003910

   This pull request **introduces 5 alerts** when merging 9ebadf429b3aef83feaac7eccfe42f3c7fd97408 into 176934e849d50a2360f50db7771c1775110308f3 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-6d0cff61024241c613e0a2be9c4e64732652057c)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007150095


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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`

Review Comment:
   yes you are absolutely correct. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007171468


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> existingJob = client.jobExists(k8sTaskId);
+                  if (!existingJob.isPresent()) {

Review Comment:
   this is from the comment earlier.  This is the scenario this happens.
   1. overlord launches task.  
   2. overlord goes down
   3. overlord comes up, calls get knownTasks() which calls run()
   4. Now the task doesn't need to be launched, it already exists, just track it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007222105


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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
+                    );

Review Comment:
   This is a weird case.  So the task reports back on completion its status.  But in case the call is missed by the overlord, or in the case when the task process just dies like a `kill -9` dead.  Then we still want to track the job, in the former case the job can still succeed and the overlord callback was missed, in the case of a kill -9 the task would never report back.  So as our last step we grab the status from the job itself. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
clintropolis commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008566692


##########
indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java:
##########
@@ -135,4 +135,24 @@ default TaskLocation getTaskLocation(String taskId)
   Map<String, Long> getLazyTaskSlotCount();
 
   Map<String, Long> getBlacklistedTaskSlotCount();
+
+  /**
+   * Beacause the k8s task runner is an extension, we need to know the task runner type in the overlord resource
+   */
+  default boolean isK8sTaskRunner()

Review Comment:
   since this class is marked `@PublicApi`, I think we should consider naming this something less specific, like `isDirectTaskRunner` or `isWorkerFreeTaskRunner` something to indicate that it is worker free but can still run up to `getTotalTaskSlotCount` tasks (based on how its being used in `OverlordResource`)



##########
services/src/test/java/org/apache/druid/cli/MainTest.java:
##########
@@ -29,9 +29,6 @@
 
 import java.util.Properties;
 
-/**
- * This test verifies injection for {@link ServerRunnable}s which are discoverable Druid servers.
- */

Review Comment:
   nit: intended change?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,129 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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 kubernetes jobs removing the need for your middle manager.  
+
+Consider this an [EXPERIMENTAL](../experimental.md) feature mostly because it has not been tested yet on a wide variety of long-running Druid clusters.
+
+## How it works
+
+The K8s extension takes the podSpec of your `Overlord` pod and creates a kubernetes job from this podSpec.  Thus if you have sidecars such as Splunk or Istio it can optionally launch a task as a K8s job.  All jobs are natively restorable, they are decoupled 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.encapsulatedTask: true`

Review Comment:
   `druid.indexer.task.encapsulatedTask` doesn't seem documented anywhere (or in javadocs), do we need to elaborate on what it does?



##########
services/src/main/java/org/apache/druid/cli/CliPeon.java:
##########
@@ -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);
+            }

Review Comment:
   nit: this also seems a bit strange and special caseish for a specific thing, though i'm not sure introducing another property here is the solution...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1005892581


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   There is and its no different from a configmap max size either.  That is why I compress and base64 encode the task.json file when i pass it in the spec. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1006252095


##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -125,6 +134,49 @@ 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);
+    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)) {

Review Comment:
   it was how the original code was handling it.  I tried to minimize changes to core, ideally a message would not determine whether to throw an exception or not.  Let me know if you have strong feelings here. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995022083


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   actually one quesiton, for env variables being configurable, do you have a usecase?  It takes all the env variables of the overlord pod running, would there be a situation where you would need something additional here?  The container shouldn't need more than what druid has to work, right? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995054151


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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....");

Review Comment:
   sure thing. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995053387


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   sure thing!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1278289541

   I noticed a slight issue when testing on heavily loaded k8s clusters.  Those that take a while to spin up / down pods.  
   
   Currently the way things work: 
   	1. The overlord launches a task, waits for the pod to come up, it not only waits for a pod ip to be available, but also opens a socket connection to the peon processes web server.  Once you have a socket connection the task is considered started.
   	2. Then it will monitor the job to complete, but this is a blocking call, after the job completes, sends pushes task reports, logs and then does a delete for the peon k8s job.
   
   What was problematic here, is I noticed that the process was complete, the task itself was finished, but the k8s operations were slow on my cluster.  The task itself took less than a minute to complete, but the entire k8s lifecycle was taking much longer around 10 minutes.  (A very heavily loaded cluster). 
   
   Thus the task status only updates in the finally block so the interval lock is not released until the status updates.  I have another patch I will test on our clusters but a brief description of the patch is this: 
   
   1. The overlord launches a task, waits for the pod to come up.  No more opening a socket for the webserver.  Instead I added 2 TaskActions: One to update the `TaskStatus`, one to update the `TaskLocation`.
   2. Now in the `AbstractTask` the setup method will update its own location before the `run` method is called.  If the overlord goes down for whatever reason, that is fine.  We don't lose anything really, the call fails and the task itself dies. 
   3. In the cleanup method of the `AbstractTask`we also update the `TaskLocation` to `unknown` and update the status to `success` or `failure`.  So when the process exits, we can give up the lock and things wont be blocked.  In case the pod goes away during the cleanup, that is okay, the overlord will still monitor the job and report the correct results albeit a bit slower.  
   
   
   I thought about this approach originally, but didn't want to make too many changes to `core`, but after running this patch on our clusters, I do think this is the best approach.  I will call it out in a separate commit so you guys can review this work, it doesn't seem too abrasive to me, but we should focus on correctness here and not hold the locks longer than necessary as we were doing in the forking task runner. 
   
   I will test this out over the next day or so and then add the commit to this PR. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995002363


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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`

Review Comment:
   That is correct, there is no mixed mode support.  If you turn on k8s mode all tasks are k8s tasks.  We have been running this patch and we don't launch any MM's.  If you deploy using the operator, this is quite easy to do as you just remove the middleManager section of the spec completely.  I will add that to the docs, good point.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1001034505


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   you have a valid point, maybe in a future PR, we could add some config map where you can define a podSpec template.  That should be pretty easy to do.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007149750


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.

Review Comment:
   No what I meant is that before you could pass javaOpts string and a javaOptsArray for your peon tasks, it was recommended to pass the array, but the string was still supported.   In this patch, we only support the array. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1263834245

   This pull request **introduces 5 alerts** when merging afff50921b672cf90ceef9ce1c5e4c5610dd6362 into ce5f55e5ce00d876277424ea0724b70a330315b3 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-16e8179dd88e3d9f89388a3f5baf2bc54d55cc4d)
   
   **new alerts:**
   
   * 3 for Uncontrolled data used in path expression
   * 2 for Unused format argument


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008535755


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.google.common.base.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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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, 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(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } 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(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");
+    cleanupExecutor.shutdownNow();
+    log.info("Stopped...");
+  }
+
+  @Override
+  public Map<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> command = new ArrayList<>();
+    // must use deep storage shuffle now with k8s peon tasks.

Review Comment:
   good point, that is left-over from something.  I'll remove it. 



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.google.common.base.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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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, 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(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } 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(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007168468


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"

Review Comment:
   Sure thing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007426146


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   yes definitely.  Right now I give you the ability to override annotations and labels.  In a later PR we can make this more configurable for users.  I have used this patch for a few druid clusters and did not need to override anything else, but that is just my experience.  I think going forward we make  this more configurable in the future.  The goal of this patch was to make it as easy as possible for users to switch from using MM's to k8s jobs.  Currently its add 4 required configurations for the overlord and you are good to go. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1006253863


##########
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 ;;

Review Comment:
   because this was an extension (that is experimental), i did not want to change the druid.sh, something that everyone uses.  That would mean changing the script for an experimental extension.  I hope one day this work goes into core, if that was the case, I would've just modified the druid.sh to handle this. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1008136709


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   For sure, I don't think we need those features right away. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1292671526

   > @abhishekagarwal87
   > 
   > > may not be an issue but is there a limit in k8s on how large the env variable value can be? The task JSON could be large.
   > 
   > I just checked the defaults for k3s, microk8s, GKE, and EKS:
   > 
   >     * A configMap value is limited to 1MiB
   > 
   >     * Similarly, an inline `env` value is limited to 1MiB
   > 
   >     * The entire final command passed to the container (command + arguments + ENVs) is limited to the `ARG_MAX` configuration of the host node, which defaults to 2MiB on Linux.
   > 
   > 
   > None of our tasks reach close to a mebibyte, but if that could be a problem, then the best would be retrieving the task from the coordinator.
   > 
   > Mounting a config map instead of passing as an env var is cleaner IMHO, but would only make a difference regarding limits if there were multiple envs smaller than 1MiB that combined go over 2MiB.
   
   I didn't want to waste quotas for no reason.  In k8s you can have a quota for configmaps.  I was attempting to not eat up quota items for this work.  Config map is cleaner, but env doesn't eat up quotas.  Also we compress and base64 encode the task.json, I doubt it would ever get large enough to cause a problem.  We have some pretty large task.json files running with this patch and they have not come close to hitting the limits.   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] Fryuni commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
Fryuni commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1292677575

   > We have some pretty large task.json files running with this patch and they have not come close to hitting the limits.
   
   Yeah, our largest task specs are only about 4KiB. I don't see any task getting close to those limits.
   
   If and when someone has a use case for a spec that is at the scale of 1MiB, there are alternatives.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r986522182


##########
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 ;;

Review Comment:
   do we need all this for a script that is meant to launch a peon? 



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   ```suggestion
   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 decoupled from the druid deployment, thus restarting pods or doing upgrades has no effect on tasks in flight.  They will continue to run and when the overlord comes back up it will start tracking them again.  
   ```



##########
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"

Review Comment:
   why is there a separate script for peon? 



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -125,6 +134,49 @@ 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);
+    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)) {

Review Comment:
   hmm. we should have a boolean flag or we can have `setup` throw a checked exception. 



##########
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)|

Review Comment:
   ```suggestion
   |kubernetes-overlord-extensions|Support for launching tasks in k8s without Middle Managers|[link](../development/extensions-contrib/k8s-jobs.md)|
   ```



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.

Review Comment:
   It should be the same namespace under which the overlord is running. 



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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"}'`

Review Comment:
   IMO the k8s indexer config should all be prefixed with `druid.indexer.runner.k8s`. 



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,127 @@
+---
+id: K8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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 kubernetes jobs removing the need for your middle manager.  
+
+Consider this an [EXPERIMENTAL](../experimental.md) feature mostly because it has not been tested yet on a wide variety of long-running Druid clusters.
+
+## How it works
+
+The K8s extension takes the podSpec of your `Overlord` pod and creates a kubernetes job from this podSpec.  Thus if you have sidecars such as Splunk or Istio it can optionally launch a task as a K8s job.  All jobs are natively restorable, they are decoupled 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.e  nableTaskLevelLogPush: true`

Review Comment:
   ```suggestion
   `druid.indexer.runner.type: K8s` and `druid.indexer.task.enableTaskLevelLogPush: true`
   ```



##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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)

Review Comment:
   may not be an issue but is there a limit in k8s on how large the env variable value can be? The task JSON could be large.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995055120


##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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<? extends TaskRunnerWorkItem> getKnownTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods()) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<ScalingStats> getScalingStats()
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public Map<String, Long> getIdleTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getUsedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getLazyTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getBlacklistedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public boolean isK8sTaskRunner()
+  {
+    return true;
+  }
+
+  @Override
+  public void unregisterListener(String listenerId)
+  {
+    for (Pair<TaskRunnerListener, Executor> pair : listeners) {
+      if (pair.lhs != null && pair.lhs.getListenerId().equals(listenerId)) {
+        listeners.remove(pair);
+        log.info("Unregistered listener [%s]", listenerId);

Review Comment:
   Sure thing. 



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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<? extends TaskRunnerWorkItem> getKnownTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods()) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<ScalingStats> getScalingStats()
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public Map<String, Long> getIdleTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getUsedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getLazyTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getBlacklistedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public boolean isK8sTaskRunner()
+  {
+    return true;
+  }
+
+  @Override
+  public void unregisterListener(String listenerId)
+  {
+    for (Pair<TaskRunnerListener, Executor> 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<TaskRunnerListener, Executor> pair : listeners) {
+      if (pair.lhs != null && pair.lhs.getListenerId().equals(listener.getListenerId())) {
+        throw new ISE("Listener [%s] already registered", listener.getListenerId());
+      }
+    }
+
+    final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
+    log.info("Registered listener [%s]", listener.getListenerId());
+    listeners.add(listenerPair);
+  }
+
+  @Override
+  @SuppressForbidden(reason = "Sets#newHashSet")
+  public Collection<TaskRunnerWorkItem> getRunningTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods(Sets.newHashSet(PeonPhase.RUNNING))) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<TaskRunnerWorkItem> 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)) {

Review Comment:
   you got it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995016716


##########
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 the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  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<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> 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<String, String> labels,
+      Map<String, String> 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<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> 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<String> javaOpts = context.getJavaOpts();
+    Optional<Long> optionalXmx = getJavaOptValueBytes("-Xmx", javaOpts);
+    long heapSize = HumanReadableBytes.parse("1g");
+    if (optionalXmx.isPresent()) {
+      heapSize = optionalXmx.get();
+    }
+    Optional<Long> 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<String> mainCommand = Lists.newArrayList("sh", "-c");
+    // update the command
+    List<Container> 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<String, Quantity> resources = ImmutableMap.of(
+        "cpu",
+        new Quantity("1000", "m"),

Review Comment:
   All tasks use a single core, I don't think there is a need to configure this value.  It would most likely lead to issues if you set it at anything but 1 cpu. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r995005949


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.

Review Comment:
   So this one was a tricky one.  Some users have quotas setup for their namespace while others do not.  Quotas such as how many jobs can i have concurrently running.  I believe if you do this, you will just have unscheduled pods while resources are not available and then when they become available they will launch.  If you use aws, you could perhaps tie in the autoscaler and have some logic stating if i have x amount of pods unscheduled then scale up ec2 instances.  And scale down the same with the reverse logic, if I have ec2 instances not using resources, scale things down. 
   
   To answer your question, yes we should always ask users to set `maxSize` less than the k8s capacity, unless they have autoscaling configured.  In that case its up to the user to determine what they can potentially scale up to. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] churromorales commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
churromorales commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1279145851

   > Hey, this is an awesome PR, thanks for all the work I am concerned about this part, though:
   > 
   > > The KubernetesTaskRunner runs in the overlord process. When it has a request to launch a task, it goes to the K8sApi, grabs its own PodSpec (the overlord itself). Takes that podSpec, modifies the necessary attributes (eg: command, labels, env variables etc). Takes the task.json, compresses and base64 encodes it. Then launches a K8s Job.
   > 
   > The resource use for overlords and tasks is vastly different, using the same podSpec for those would require reserving much more resources for the overlords and the tasks than necessary.
   > 
   > One idea is that instead of reading from the overlord pod itself, it would be better to provide the name of a config map containing the podSpec.
   > 
   > Another idea: the middle-managers have tiers, which we can use to allocate tasks to middle-managers of different tiers. Those tiers could be used to define different podSpecs for different tasks without changing the tasks API.
   > 
   > Those changes can be done on a later PR. All in all, this PR is a huge step forward in making Druid more scalable for cloud environments.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] Fryuni commented on pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
Fryuni commented on PR #13156:
URL: https://github.com/apache/druid/pull/13156#issuecomment-1279592919

   >This doesn't use the resources of the overlord, we overwrite the resources for podSpec. The memory is derived from your JAVA_OPTS (for your task) basically 1.2*(Xmx + Dbb) for the memory limit and the cpu is always 1.
   
   That is super great news on the memory side, but not so great on the CPU side for us. We have 23 continuously running ingestion tasks and they only use a tenth of a core (100m) each, so fixing the resource request to one core is a 9x overprovision. A full core works for our compaction tasks, but those run for about 10 minutes every 4 hours.
   
   We'll explore some things with Druid in the near future, so we might use this PR before it lands and give some feedback.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r994031451


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   Thanks for this doc. Super useful for us reviewers.
   
   Swap the above two paragraphs: Explain what this is before we note it is experimental?



##########
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.

Review Comment:
   This is similar to the standard script. Is it a copy? If a copy, then perhaps we can avoid an actual copy: add a rule to the `distribution` project (likely in `assembly.xml`) to copy the existing script to `peon.sh` where needed.
   
   If this version has changes, can they be applied to the standard script somehow to avoid the copy?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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`

Review Comment:
   Explain a bit? With K8s, there is no MM, right? So, under K8s, this setting is mandatory?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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|

Review Comment:
   Should the K8s-specific items be under `druid.indexer.k8s`?



##########
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

Review Comment:
   Mighty simple config! Often, the `Config` suffix means a JSON configuration object. Is there anything to put here, or is this a vestige of a path not taken? Else, a comment would be helpful.



##########
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.

Review Comment:
   What happens if the task exists while OL is down or failing over? Do we capture logs, reports & completion status somehow? Or is the independent-run valid only while the task executes?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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|
+

Review Comment:
   Would be great to have an example or tutorial for how to set this up. Probably too much of an ask for this PR (let's get the code in, then do the docs.) I'm thinking:
   
   * How to configure Druid for your K8s cluster.
   * A typical Peon pod spec.
   * How to launch a job.
   * How to monitor the job in both Druid and K8s.
   
   Doing the above will raise additional questions/details because both K8s and Druid are quite fiddly at the detail level.



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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);
+                    }

Review Comment:
   Nit: simplify flow?
   
   ```java
                       if (job.getStatus().getActive() != null) {
                         // the job is active lets monitor it
                         completedPhase = monitorJob(task, k8sTaskId);
                       } else if (job.getStatus().getSucceeded() != null) {
                         completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
                       } else {
                         completedPhase = new JobResponse(job, PeonPhase.FAILED);
                       }
   ```



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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
+                    );

Review Comment:
   These two blocks handle the completed/failed status we just set above. Can the blocks be merged into the prior blocks? Or, will we come this way more than once if the job ends?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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)

Review Comment:
   Might it be a bit easier to follow if, when moving the "controller" to a separate class, these methods were members of that new class? These seem more related to the controller (which would know the pod, task and K8s task id) than to the runner (which, it seems, is the thing that launches the controller.)



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   It -> "The K8s extension"
   
   Please capitalize proper nouns.



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.

Review Comment:
   In other words, the extension will run K8s jobs up to the limit specified? The assumption is that K8s has at least this much capacity? What happens if I configure the `maxSize` to be greater than my K8s capacity? Will the "extra" Druid tasks remain queued?
   
   Druid recently introduced MSQ, which launches a controller, then launches a wave of workers. Could we get into a deadlock situation if K8s has enough capacity to launch only half the workers? If so, we should note to users to set `maxSize` to less than the K8s capacity.



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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`

Review Comment:
   From this, can I infer that, with this extension, all tasks run in K8s? There is no mixed mode where I can choose the hosting type per-task? (Not sure mixed usage is useful, just asking.)
   
   If use of this extension is binary, then perhaps note somewhere that the cluster should be configured with no MiddleManagers and no Indexers. (If that is, in fact, what I should do.)



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.

Review Comment:
   When we say "no longer support", do we mean "not supported by the K8s extension"?
   
   What does it mean to "be moved under the overlord"? Where we had `druid.indexer.foo` we now have `druid.overlord.foo`?
   
   If this is so, and those properties only apply when K8s is in effect, should the config root perhaps be `druid.k8s` or some such?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.
+- For the sidecar support to work, your entrypoint / command in docker must be explicitly defined your spec.  
+
+You can't have something like this: 
+Dockerfile: 
+``` ENTRYPOINT: ["foo.sh"] ```
+
+and in your sidecar specs: 
+``` container:
+        name: foo
+        args: 
+           - arg1
+           - arg2 
+```
+
+That will not work, because we cannot decipher what your command is, the extension needs to know it explicitly. 
+**Even for sidecars like isito which are dynamically created by the service mesh, this needs to happen.* 

Review Comment:
   Mis-matched `**`/`*`



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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....");

Review Comment:
   Debug? Here and above? Probably not very useful for the typical Druid user...



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.  

Review Comment:
   How is "your role" communicated to Druid? Via a config? Something else?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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));

Review Comment:
   Should these ports be hardcoded? Which ports are these? The listen ports on the pod which the OL uses? Or, the listen ports on Druid which the K8s peon uses? If for OL, should we use the configured ports? If for the pod, can we guarantee these ports are always available?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   There seems to be one pod spec for a cluster. Should I then ensure that the pod is large enough for my largest task? Would a future enhancement be the ability to override the pod spec per-task so that light-weight tasks (the MSQ controller) takes less resources than a heavy-weight task (an MSQ worker)?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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");

Review Comment:
   General question: if a peon supports queries, it will have incoming connections. How do we ensure that the Peon's IP address is visible to the Broker? K8s usually has some kind of overlay network. Are we assuming Druid runs within K8s so it is on that same network? Or, that somewhere we've set up a proxy into the overlay network? It's been a while since I used K8s, perhaps the proxy isn't needed? Or, the networks are merged in AWS? Is that true on plain-Jane K8s?



##########
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<String, TaskRunnerFactory> 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<String, TaskLogs> 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<? extends Module> getJacksonModules()
+  {
+    return Collections.emptyList();
+  }

Review Comment:
   Nit: the above boilerplate is no longer needed. There is a default method for the no-Jackson-modules case.



##########
extensions-contrib/kubernetes-overlord-extensions/pom.xml:
##########
@@ -0,0 +1,233 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.druid.extensions.contrib</groupId>
+  <artifactId>druid-kubernetes-overlord-extensions</artifactId>
+  <name>druid-kubernetes-overlord-extensions</name>
+  <description>druid-kubernetes-overlord-extensions</description>
+
+  <parent>
+    <groupId>org.apache.druid</groupId>
+    <artifactId>druid</artifactId>
+    <version>25.0.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-server</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.squareup.okhttp3</groupId>
+          <artifactId>okhttp</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.squareup.okhttp3</groupId>
+          <artifactId>logging-interceptor</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-processing</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-indexing-service</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>2.6</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>2.11.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>3.12.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-common</artifactId>
+      <version>4.1.68.Final</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <version>1.26.0</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-text</artifactId>
+      <version>1.9</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-model-core</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+    <dependency>
+      <groupId>javax.validation</groupId>
+      <artifactId>validation-api</artifactId>
+      <version>1.1.0.Final</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-model-batch</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-client</artifactId>
+      <version>5.12.2</version>
+    </dependency>
+
+    <!-- Tests -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.fabric8</groupId>
+      <artifactId>kubernetes-server-mock</artifactId>
+      <version>5.12.2</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-api</artifactId>
+      <version>5.8.2</version>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- others -->
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject.extensions</groupId>
+      <artifactId>guice-multibindings</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.9.5</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-indexing-service</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-server</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-processing</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+</project>

Review Comment:
   Nit: missing newline.



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.disableClientProxy`|`boolean`|Use this if you have a global http(s) proxy and you wish to bypass it.|false|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.

Review Comment:
   Where is this name space specified? I didn't see an option for it above.



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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"

Review Comment:
   resonable -> reasonable
   
   Actually, leave off ", set it to a resonable value" as that is generally implied and not super helpful to a reader.



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {

Review Comment:
   Suggestion: the block that follows is a rather complex (and critical) job state machine. It would be a bit easier to read if it were packed in its own class so a reader can focus on just the controller bits without the surrounding clutter.



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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 {

Review Comment:
   ```java
   // Job exists. Check if it is done, failed or still running.
   ```



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());

Review Comment:
   Can we simplify to:
   
   ```java
   return client.getPeonLogs(new K8sTaskId(taskid));
   ```



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> existingJob = client.jobExists(k8sTaskId);
+                  if (!existingJob.isPresent()) {

Review Comment:
   Suggestion: a few comments would help the reader. Here is this true?
   ```java
   // Job does not yet exist. Create it. We'll poll again later to check progress.
   ```
   



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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);

Review Comment:
   Did we create a status for the job-started or job-running paths?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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<? extends TaskRunnerWorkItem> getKnownTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods()) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<ScalingStats> getScalingStats()
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public Map<String, Long> getIdleTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getUsedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getLazyTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getBlacklistedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public boolean isK8sTaskRunner()
+  {
+    return true;
+  }
+
+  @Override
+  public void unregisterListener(String listenerId)
+  {
+    for (Pair<TaskRunnerListener, Executor> pair : listeners) {
+      if (pair.lhs != null && pair.lhs.getListenerId().equals(listenerId)) {
+        listeners.remove(pair);
+        log.info("Unregistered listener [%s]", listenerId);

Review Comment:
   Debug? Here and similar items below?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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);

Review Comment:
   It seems that we dedicate one Overlord thread to each K8s task? That this thread is, essentially, the OL proxy for the K8s task? If so, perhaps we can describe this design in a comment.



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   The above says "pod spec". Is that literally what we need? Or, do we provide a [pod template](https://kubernetes.io/docs/concepts/workloads/pods/)? Per the docs,
   
   > Controllers for [workload](https://kubernetes.io/docs/concepts/workloads/) resources create Pods from a pod template and manage those Pods on your behalf.
   >
   > PodTemplates are specifications for creating Pods, and are included in workload resources such as [Deployments](https://kubernetes.io/docs/concepts/workloads/controllers/deployment/), [Jobs](https://kubernetes.io/docs/concepts/workloads/controllers/job/), and [DaemonSets](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/).
   
   Would I be on the right track to think of OL as a _controller_ in this context?
   
   Or, does this extension use the pod spec as the template (because, say, the other template properties don't apply here)?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -0,0 +1,125 @@
+---
+id: k8s-jobs
+title: "MM-less Druid in K8s"
+---
+
+<!--
+  ~ 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.
+  -->
+
+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.  

Review Comment:
   Perhaps provide just a bit more background for newbies. Where does the pod spec come from? Something like:
   
   > With this extension, your Overload tasks run in a Kubernetes pod. You define that pod using a pod spec. The pod spec names a Docker image with Druid installed, identifies K8s settings, provides Druid configuration, passes in secrets, etc. You create the pod spec outside of Druid. We suggest you test the spec directly in K8s before using it with Druid.
   
   I'm sure some (all?) of the above is wrong: it just identifies what I, as a newbie to this extension, would need to know. Focus in particular on anything Druid-specific I need to provide in the pod spec.
   
   Also, where do I put the pod spec? In K8s somewhere? On the local disk of each Druid node?



##########
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> 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<Pod, Job> 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<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> 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<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> 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<String> generateCommand(Task task)
+  {
+    final List<String> 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<? extends TaskRunnerWorkItem> getKnownTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods()) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<ScalingStats> getScalingStats()
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public Map<String, Long> getIdleTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getUsedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getLazyTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> getBlacklistedTaskSlotCount()
+  {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public boolean isK8sTaskRunner()
+  {
+    return true;
+  }
+
+  @Override
+  public void unregisterListener(String listenerId)
+  {
+    for (Pair<TaskRunnerListener, Executor> 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<TaskRunnerListener, Executor> pair : listeners) {
+      if (pair.lhs != null && pair.lhs.getListenerId().equals(listener.getListenerId())) {
+        throw new ISE("Listener [%s] already registered", listener.getListenerId());
+      }
+    }
+
+    final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
+    log.info("Registered listener [%s]", listener.getListenerId());
+    listeners.add(listenerPair);
+  }
+
+  @Override
+  @SuppressForbidden(reason = "Sets#newHashSet")
+  public Collection<TaskRunnerWorkItem> getRunningTasks()
+  {
+    List<TaskRunnerWorkItem> result = new ArrayList<>();
+    for (Pod existingTask : client.listPeonPods(Sets.newHashSet(PeonPhase.RUNNING))) {
+      try {
+        Task task = adapter.toTask(existingTask);
+        ListenableFuture<TaskStatus> 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<TaskRunnerWorkItem> 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)) {

Review Comment:
   `switch`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org