You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "churromorales (via GitHub)" <gi...@apache.org> on 2023/02/14 18:56:41 UTC

[GitHub] [druid] churromorales opened a new pull request, #13804: Upgrade the fabric client to support newer versions of k8s

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

   This PR contains 2 items. 
   
   1. Upgrade the k8s client to support newer versions of k8s. 
   2. Ability to override specific Metrics Monitors 
    
   We found that when we used the `TaskCountStatsMonitor` in the overlord config, the peon tasks would not start because they inherit the monitors from the parent process, which used to be the Middle Manager (but that would never have that monitor originally).  So now you can override this value with the following config: 
   
   `druid.indexer.runner.peonMonitors`


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120733527


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   that could work, want me to change it?  This definitely works above, I don't have access to many different k8s environments, and someone in the community helped me test the above on 1.25.  I could definitely change this, but I unfortunately have no way of testing this out. 



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1450327032

   @churromorales - can you address the build failures? 


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1487385805

   > 
   
   it is a good question.  I think the way I coded it was not very clear, I apologize for that. 
   I tried to make the `KubernetesTaskRunnerTest` cover the different scenarios.  
   
   Lets look at this case....
   
   startup overlord, 
   1st thing it does is go to k8s and see if there are any running tasks.  If there are tasks, watch them and do bookkeeping when they finish. 
   
   The conditional block you are referring to handles that case.   It first does a check to see if the task is not active, then checks the state.  If it is still running it does the `if` statement you suggested (in following lines to monitorJob())`, so no need to do it twice.    It's just a short circuit in case its already finished, grab the status and return, no need to make more k8s calls.
   
   Does that help make sense of how things are working?  These are good questions, and I want to make sure the code is doing the correct 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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120737472


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClientTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.Client;
+import io.fabric8.kubernetes.client.Config;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+class DruidKubernetesClientTest
+{
+
+  @Test
+  void makingCodeCoverageHappy()

Review Comment:
   So I had tests for a class, then i upgraded a client library and had to change some things.  but the functions were all tested before.  Now i would expect that if tests pass I would be good, since it was already tested before.  But turns out if you have to make any changes to make the new library apis work, then code coverage complains.  It is unfortunate here, because I didn't change any behavior. 



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1112268989


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   This was a tricky one.  For some background 
   
   the overlord launches a task, waits for it to complete and then returns the status.  Now in k8s < 1.25 it would just mean once the pod is not active, go grab the status.
   Now the contact has changed a bit, because in 1.25 they introduced finalizers:
   https://github.com/kubernetes/kubernetes/pull/110948
   
   And we noticed that when we ran tasks on a 1.25 k8s  druid cluster they would complete fine but marked as failure.
   We outputted the job status and noticed that the job was not active, but both success and failure were not set, but there was this field that had
   ```
   uncountedTerminatedPods=UncountedTerminatedPods(failed=[], succeeded=[e916cbf9-467a-45f3-86a7-3767145d6384], additionalProperties={})
   ```
   which from the docs:
   ```
   UncountedTerminatedPods holds the UIDs of Pods that have terminated but the job controller hasn't yet accounted for in the status counters. The job controller creates pods with a finalizer. When a pod terminates (succeeded or failed), the controller does three steps to account for it in the job status: (1) Add the pod UID to the arrays in this field. (2) Remove the pod finalizer. (3) Remove the pod UID from the arrays while increasing the corresponding counter. This field is beta-level. The job controller only makes use of this field when the feature gate JobTrackingWithFinalizers is enabled (enabled by default). Old jobs might not be tracked using this field, in which case the field remains null.
   ```
   So now what happens is the job goes from a state where it is not active, to having uncountedTerminatedPods to then having a status with success or failure.  I will push up a one-line fix to make this work, but for those of you working with 1.25 version of k8s, I’m sure you will be affected as well.
   
   Basically add another check to wait on,
   Right now we wait for this:
   ```
   // block until
   job.getStatus() != null && job.getActive() == null
   // then return 
   return job.getStatus().getSucceeded() != null
   ```
   So the change has to become
   ```
   // block until
   job.getStatus() != null && job.getActive() == null && (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() !=null)
   // then return 
   return job.getStatus().getSucceeded() != null
   ```
   This should keep things backwards compatible and working in all versions of k8s



-- 
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 closed pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis closed pull request #13804: Upgrade the fabric client to support newer versions of k8s
URL: https://github.com/apache/druid/pull/13804


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales closed pull request #13804: Upgrade the fabric client to support newer versions of k8s
URL: https://github.com/apache/druid/pull/13804


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1113646039


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   @abhishekagarwal87 what do you think?  This should bring the client up-to-date as well as fix things for those druid users running on k8s 1.25+ that were having issues.
   



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1113850734


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -121,12 +121,12 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
   public boolean cleanUpJob(K8sTaskId taskId)
   {
     if (!debugJobs) {
-      Boolean result = clientApi.executeRequest(client -> client.batch()
-                                                                .v1()
-                                                                .jobs()
-                                                                .inNamespace(namespace)
-                                                                .withName(taskId.getK8sTaskId())
-                                                                .delete());
+      Boolean result = clientApi.executeRequest(client -> !client.batch()
+                                                                 .v1()
+                                                                 .jobs()
+                                                                 .inNamespace(namespace)
+                                                                 .withName(taskId.getK8sTaskId())
+                                                                 .delete().isEmpty());

Review Comment:
   what does `delete` return here? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java:
##########
@@ -54,6 +54,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 // must have a kind / minikube cluster installed and the image pushed to your repository
+@Disabled

Review Comment:
   We are using K3S now. do you still same issues? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   So for the older k8s version, we can expect that when job.getActive() is null, one of `success` or `failure` field must be set. Question - do we need to check for getActive() result at all? can we just block till the job either succeeded or failed? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClientTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.Client;
+import io.fabric8.kubernetes.client.Config;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+class DruidKubernetesClientTest
+{
+
+  @Test
+  void makingCodeCoverageHappy()

Review Comment:
   Hmm. This is a bit of a bummer. Is there no way to avoid this? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java:
##########
@@ -205,4 +207,51 @@ void testNoPrimaryFound()
     });
   }
 
+  @Test
+  void testAddingMonitors() throws IOException
+  {
+    TestKubernetesClient testClient = new TestKubernetesClient(client);
+    PeonCommandContext context = new PeonCommandContext(
+        new ArrayList<>(),
+        new ArrayList<>(),
+        new File("/tmp/")
+    );
+    KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
+    config.namespace = "test";
+    K8sTaskAdapter adapter = new SingleContainerTaskAdapter(testClient, config, jsonMapper);
+    Task task = K8sTestUtils.getTask();
+    // no monitor in overlord, no monitor override
+    Container container = new ContainerBuilder()
+        .withName("container").build();
+    adapter.addEnvironmentVariables(container, context, task.toString());
+    assertFalse(container.getEnv().stream().anyMatch(x -> x.getName().equals("druid_monitoring_monitors")));

Review Comment:
   Please print the container.getEnv if the assertion fails. 



-- 
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 pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1498490120

   closing since these changes were added to #14028


-- 
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] techdocsmith commented on a diff in pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "techdocsmith (via GitHub)" <gi...@apache.org>.
techdocsmith commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1123727246


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -66,6 +66,7 @@ Additional Configuration
 |`druid.indexer.runner.javaOptsArray`|`JsonArray`|java opts for the task.|`-Xmx1g`|No|
 |`druid.indexer.runner.labels`|`JsonObject`|Additional labels you want to add to peon pod|`{}`|No|
 |`druid.indexer.runner.annotations`|`JsonObject`|Additional annotations you want to add to peon pod|`{}`|No|
+|`druid.indexer.runner.peonMonitors`|`JsonArray`|Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the overlord.|`[]`|No|

Review Comment:
   ```suggestion
   |`druid.indexer.runner.peonMonitors`|`JsonArray`|Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord.|`[]`|No|
   ```
   nit. capitalize Druid service names.



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120735806


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java:
##########
@@ -54,6 +54,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 // must have a kind / minikube cluster installed and the image pushed to your repository
+@Disabled

Review Comment:
   k3s?  This was for anyone who had an existing k8s cluster, they could just run this test and point it there.  The k8s integration tests are definitely not flexible enough for me to test mm-less without rewriting a lot of 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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1125113300


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   @nlippis I believe you can test by taking this commit and making that small change and apply it onto the 25 branch.  I think if it works for you, ill resubmit this with the change. 



-- 
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] mindreader commented on pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "mindreader (via GitHub)" <gi...@apache.org>.
mindreader commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1478819088

   Or let me know if I misunderstood you.
   
   
   
   
   > > Do you also need to make this change?
   > > ```diff
   > > --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   > > +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   > > @@ -180,7 +180,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
   > >                      completedPhase = monitorJob(peonPod, k8sTaskId);
   > >                    } else {
   > >                      Job job = existingJob.get();
   > > -                    if (job.getStatus().getActive() == null) {
   > > +                    if (job.getStatus() != null && job.getStatus().getActive() == null && (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() !=null)) {
   > >                        if (job.getStatus().getSucceeded() != null) {
   > >                          completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
   > >                        } else {
   > > ```
   > 
   > I believe that is the crux of the change. I explain why here: [#13804 (comment)](https://github.com/apache/druid/pull/13804#discussion_r1112268989) Or let me know if I misunderstood you.
   
   Sorry I wasn't clear. There are two spots in the code that have that same success / failure logic, I only see one in this pr, after the job is monitored. But I also see one before the job is monitored just after it is spawned in KubernetesTaskRunner. If the logic is not necessary at that point, please feel free to ignore me.


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1489675995

   I understand @churromorales. would you be open to giving write access on your branch to someone who wants to take this PR forward? 


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1106550333


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/LogWatchInputStream.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.dsl.LogWatch;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LogWatchInputStream extends InputStream

Review Comment:
   sure
   



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1106642496


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/LogWatchInputStream.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.dsl.LogWatch;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LogWatchInputStream extends InputStream

Review Comment:
   I believe the issue with testing is invalid.  I just upgraded a client library.  I did have to change code but I did not change method contracts or behavior so existing tests should all pass.  I don’t think I can add any meangful tests here without just eating up cpu cycles.  



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1489512690

   @abhishekagarwal87 The test failing are from the other PR that was merged, I don't have cycles to look and fix whatever is wrong and we honestly forked our druid so whether it goes upstream is not a big deal.  
   
   I think the thing is the other PR is using all the old libraries, and it can't deserialize the pod template anymore.  So if you want to update this feature to work with newer k8s versions, that will have to be sorted out at some point, or you can just close off this PR and worry about upgrades when the time comes.  As it stands the mm-less wont work in newer versions of k8s so, I'll leave it up to imply what they want 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 pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1467126686

   @abhishekagarwal87 you good with this now?  I explained about the test coverage in an earlier comment. 


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1439187321

   @abhishekagarwal87 what do you think?  This should bring the client up-to-date as well as fix things for those druid users running on k8s 1.25+ that were having issues.
   


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1133060169


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java:
##########
@@ -54,6 +54,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 // must have a kind / minikube cluster installed and the image pushed to your repository
+@Disabled

Review Comment:
   how was this running before? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClientTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.Client;
+import io.fabric8.kubernetes.client.Config;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+class DruidKubernetesClientTest
+{
+
+  @Test
+  void makingCodeCoverageHappy()

Review Comment:
   aren't there any tests covering that path? 



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1454241784

   > @churromorales - can you address the build failures?
   I addressed them in a comment above, but maybe it was missed. 
   
   So I had tests for the `DruidKubernetesPeonClient` class, then i upgraded a client library and had to change some things. but the functions were all tested before. Now i would expect that if tests pass I would be good, since it was already tested before. But turns out if you have to make any changes to make the new library apis work, then code coverage complains. It is unfortunate here, because I didn't change any behavior.  I can add more tests, but they wont be meaningful here, I believe the build problems are all because I updated the library and had to make some code changes to make it work.  There are no logic changes here, except for the issue in 1.25, which I did test. 
   


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120743233


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -256,6 +256,7 @@ JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
   @Override
   public void updateStatus(Task task, TaskStatus status)
   {
+    log.info("Updating status %s", status);

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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1143862150


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClientTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.Client;
+import io.fabric8.kubernetes.client.Config;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+class DruidKubernetesClientTest
+{
+
+  @Test
+  void makingCodeCoverageHappy()

Review Comment:
   i added a getter to a class, the code coverage tool complained that i added new code without a test.  So I added a test for the getter.  That was new code, so i guess its okay to add a test if that's what druid wants.  Although I feel tests like this are not really that useful and make the whole suite slow in general.  



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1478413747

   > Do you also need to make this change?
   > 
   > ```diff
   > --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   > +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   > @@ -180,7 +180,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
   >                      completedPhase = monitorJob(peonPod, k8sTaskId);
   >                    } else {
   >                      Job job = existingJob.get();
   > -                    if (job.getStatus().getActive() == null) {
   > +                    if (job.getStatus() != null && job.getStatus().getActive() == null && (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() !=null)) {
   >                        if (job.getStatus().getSucceeded() != null) {
   >                          completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
   >                        } else {
   > ```
   
   I believe that is the crux of the change.  I explain why here: https://github.com/apache/druid/pull/13804#discussion_r1112268989  Or let me know if I misunderstood 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] abhishekagarwal87 commented on pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1480845961

   @a2l007 - Sorry if I am missing something but if CC is failing anyway, there is no need to add `makingCodeCoverageHappy()` test in this PR. That's the only comment I have. LGTM otherwise. 


-- 
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] github-code-scanning[bot] commented on a diff in pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1106624224


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/LogWatchInputStreamTest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.dsl.LogWatch;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class LogWatchInputStreamTest

Review Comment:
   ## Unused classes and interfaces
   
   Unused class: LogWatchInputStreamTest is not referenced within this codebase. If not used as an external API it should be removed.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4269)



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120735101


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -121,12 +121,12 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
   public boolean cleanUpJob(K8sTaskId taskId)
   {
     if (!debugJobs) {
-      Boolean result = clientApi.executeRequest(client -> client.batch()
-                                                                .v1()
-                                                                .jobs()
-                                                                .inNamespace(namespace)
-                                                                .withName(taskId.getK8sTaskId())
-                                                                .delete());
+      Boolean result = clientApi.executeRequest(client -> !client.batch()
+                                                                 .v1()
+                                                                 .jobs()
+                                                                 .inNamespace(namespace)
+                                                                 .withName(taskId.getK8sTaskId())
+                                                                 .delete().isEmpty());

Review Comment:
   ```List<StatusDetails> delete();```  used to return true or false i believe. 



-- 
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] mindreader commented on pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "mindreader (via GitHub)" <gi...@apache.org>.
mindreader commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1470028218

   Do you also need to make this change?
   
   ```diff
   --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
   @@ -180,7 +180,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
                        completedPhase = monitorJob(peonPod, k8sTaskId);
                      } else {
                        Job job = existingJob.get();
   -                    if (job.getStatus().getActive() == null) {
   +                    if (job.getStatus() != null && job.getStatus().getActive() == null && (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() !=null)) {
                          if (job.getStatus().getSucceeded() != null) {
                            completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
                          } else {
   ```


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "a2l007 (via GitHub)" <gi...@apache.org>.
a2l007 commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1478552620

   @abhishekagarwal87 Are there any additional concerns with this PR apart from the code coverage failure? From the discussion it looks like there isn't really a way to make the coverage tool pass without adding non-meaningful tests.


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1113848255


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -256,6 +256,7 @@ JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
   @Override
   public void updateStatus(Task task, TaskStatus status)
   {
+    log.info("Updating status %s", status);

Review Comment:
   this log line doesn't really include any task-specific info. can you add that too? 



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1489100193

   @abhishekagarwal87 I pushed up the fix, there were some conflicts due to something else getting merged before this.  
   Now there are issues  downloading node, I just want to let you know, I don't have anymore cycles to follow up on this patch.  If there are any more changes, feel free to take over and make whatever changes to merge if you want, or if you don't want that is fine too.  I just wont be following up on this anymore.    


-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "a2l007 (via GitHub)" <gi...@apache.org>.
a2l007 commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1112253214


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   Why is this change needed?



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "a2l007 (via GitHub)" <gi...@apache.org>.
a2l007 commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1106410985


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/LogWatchInputStream.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.dsl.LogWatch;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LogWatchInputStream extends InputStream

Review Comment:
   Can we add javadocs for this class?



##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -66,6 +66,7 @@ Additional Configuration
 |`druid.indexer.runner.javaOptsArray`|`JsonArray`|java opts for the task.|`-Xmx1g`|No|
 |`druid.indexer.runner.labels`|`JsonObject`|Additional labels you want to add to peon pod|`{}`|No|
 |`druid.indexer.runner.annotations`|`JsonObject`|Additional annotations you want to add to peon pod|`{}`|No|
+|`druid.indexer.runner.peonMonitors`|`JsonArray`|An override for the `druid.monitoring.monitors`, For the situation you have monitors setup, and do not want to inherit those from the overlord.|`[]`|No|

Review Comment:
   Nit: It would be useful to mention this property in the [extension specific docs](https://druid.apache.org/docs/latest/development/extensions-contrib/k8s-jobs.html) 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] ektravel commented on a diff in pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "ektravel (via GitHub)" <gi...@apache.org>.
ektravel commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1119053790


##########
docs/development/extensions-contrib/k8s-jobs.md:
##########
@@ -66,6 +66,7 @@ Additional Configuration
 |`druid.indexer.runner.javaOptsArray`|`JsonArray`|java opts for the task.|`-Xmx1g`|No|
 |`druid.indexer.runner.labels`|`JsonObject`|Additional labels you want to add to peon pod|`{}`|No|
 |`druid.indexer.runner.annotations`|`JsonObject`|Additional annotations you want to add to peon pod|`{}`|No|
+|`druid.indexer.runner.peonMonitors`|`JsonArray`|An override for the `druid.monitoring.monitors`, For the situation you have monitors setup, and do not want to inherit those from the overlord.|`[]`|No|

Review Comment:
   ```suggestion
   |`druid.indexer.runner.peonMonitors`|`JsonArray`|Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the overlord.|`[]`|No|
   ```



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1120747812


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java:
##########
@@ -205,4 +207,51 @@ void testNoPrimaryFound()
     });
   }
 
+  @Test
+  void testAddingMonitors() throws IOException
+  {
+    TestKubernetesClient testClient = new TestKubernetesClient(client);
+    PeonCommandContext context = new PeonCommandContext(
+        new ArrayList<>(),
+        new ArrayList<>(),
+        new File("/tmp/")
+    );
+    KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
+    config.namespace = "test";
+    K8sTaskAdapter adapter = new SingleContainerTaskAdapter(testClient, config, jsonMapper);
+    Task task = K8sTestUtils.getTask();
+    // no monitor in overlord, no monitor override
+    Container container = new ContainerBuilder()
+        .withName("container").build();
+    adapter.addEnvironmentVariables(container, context, task.toString());
+    assertFalse(container.getEnv().stream().anyMatch(x -> x.getName().equals("druid_monitoring_monitors")));

Review Comment:
   sure



-- 
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] nlippis commented on a diff in pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "nlippis (via GitHub)" <gi...@apache.org>.
nlippis commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1121778946


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java:
##########
@@ -106,13 +104,15 @@ public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit
                       .inNamespace(namespace)
                       .withName(taskId.getK8sTaskId())
                       .waitUntilCondition(
-                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null,
+                          x -> x != null && x.getStatus() != null && x.getStatus().getActive() == null
+                          && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() !=null),

Review Comment:
   Please make this change, we can test it out on a v1.25 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] churromorales commented on a diff in pull request #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "churromorales (via GitHub)" <gi...@apache.org>.
churromorales commented on code in PR #13804:
URL: https://github.com/apache/druid/pull/13804#discussion_r1136106990


##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java:
##########
@@ -54,6 +54,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 // must have a kind / minikube cluster installed and the image pushed to your repository
+@Disabled

Review Comment:
   I had a kind cluster running locally, it was in the original patch.  For anyone that wants to test with a local k8s cluster I left this test in the codebase.  If you want me to remove it, I'll be happy to get rid of it.  I find it useful for when I make changes, since I can't integration test this with the way current druid-it tests are setup. 



-- 
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 #13804: Upgrade the fabric client to support newer versions of k8s

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13804:
URL: https://github.com/apache/druid/pull/13804#issuecomment-1488115697

   @churromorales - Thank you for resolving the conflicts. There are build failures. can you take a look? 


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