kfaraz commented on code in PR #18599:
URL: https://github.com/apache/druid/pull/18599#discussion_r2591380278
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java:
##########
@@ -308,13 +307,11 @@ public void test_runKafkaSupervisor()
Assertions.assertEquals("RUNNING", supervisorStatus.getState());
Assertions.assertEquals(topic, supervisorStatus.getSource());
- // Get the task statuses
- List<TaskStatusPlus> taskStatuses = ImmutableList.copyOf(
- (CloseableIterator<TaskStatusPlus>)
- cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(null,
dataSource, 1))
- );
- Assertions.assertFalse(taskStatuses.isEmpty());
- Assertions.assertEquals(TaskState.RUNNING,
taskStatuses.get(0).getStatusCode());
+ // Confirm tasks are being created and running
+ // This more forgiving assertion avoids weird race conditions with super
specific checks like "exactly 1 running task"
Review Comment:
Yes, this is cleaner! I think we can remove this comment though.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java:
##########
@@ -70,22 +72,44 @@ public KubernetesTaskRunnerFactory(
@Override
public KubernetesTaskRunner build()
{
- KubernetesPeonClient peonClient;
- if
(adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType()))
{
- peonClient = new KubernetesPeonClient(
- druidKubernetesClient,
- kubernetesTaskRunnerConfig.getNamespace(),
- kubernetesTaskRunnerConfig.getOverlordNamespace(),
- kubernetesTaskRunnerConfig.isDebugJobs(),
- emitter
- );
+ AbstractKubernetesPeonClient peonClient;
+ boolean enableCache =
kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers();
+ boolean useOverlordNamespace =
adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType());
+
+ if (enableCache) {
+ if (useOverlordNamespace) {
+ peonClient = new CachingKubernetesPeonClient(
+ druidKubernetesClient,
+ kubernetesTaskRunnerConfig.getNamespace(),
+ kubernetesTaskRunnerConfig.getOverlordNamespace(),
+ kubernetesTaskRunnerConfig.isDebugJobs(),
+ emitter
+ );
+ } else {
+ peonClient = new CachingKubernetesPeonClient(
Review Comment:
For brevity, let's just use a single constructor and pass the overlord
namespace as null when `useOverlordNamespace` is `false`. The constructor can
then default it to empty string or leave null, whichever is appropriate.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java:
##########
@@ -80,6 +80,13 @@ public class KubernetesTaskRunnerConfig
// disable using http / https proxy environment variables
private boolean disableClientProxy;
+ @JsonProperty
+ // enable using kubernetes informer cache for peon client operations
+ private boolean enableKubernetesClientSharedInformers = false;
+
+ @JsonProperty
+ private Period kubernetesClientInformerResyncPeriod = new Period("PT5M");
Review Comment:
```suggestion
private Period k8sSharedInformerSyncPeriod = new Period("PT5M");
```
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A KubernetesPeonClient implementation that uses shared informers to read
Job and Pod state from a local cache.
+ * <p>
+ * This implementation greatly reduces load on the Kubernetes API server by
centralizing watches and allowing
+ * tasks to query cached resource state instead of making per-task API calls.
Mutable operations (job creation,
+ * deletion) still contact the API server directly.
+ * </p>
+ */
+public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient
+{
+ protected static final EmittingLogger log = new
EmittingLogger(CachingKubernetesPeonClient.class);
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ String overlordNamespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, overlordNamespace, debugJobs, emitter);
+ }
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, "", debugJobs, emitter);
+ }
+
+ @Override
+ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong,
TimeUnit unit)
+ {
+ long timeoutMs = unit.toMillis(howLong);
+ long startTime = System.currentTimeMillis();
+
+ // Give the informer 2 resync periods to see the job. if it isn't seen by
then, we assume the job was canceled.
+ // This is to prevent us from waiting for entire max job runtime on a job
that was canceled before it even started.
+ long jobMustBeSeenBy = startTime +
(clientApi.getInformerResyncPeriodMillis() * 2);
+ boolean jobSeenInCache = false;
+
+ // Set up to watch for job changes
+ CompletableFuture<Job> jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+
+ // We will loop until the full timeout is reached if the job is seen in
cache. If the job does not show up in the cache we will exit earlier.
+ // In this loop we first check the cache to see if our job is there and
complete. This avoids missing notifications that happened before we set up the
watch.
+ // If the job is not complete we wait for a notification of a job change
or a timeout.
+ // If it is a timeout, we loop back to check the cache again.
+ // If it is a job change notification, we check the job state and exit if
complete, or loop again if still running.
+ do {
+ try {
+ Optional<Job> maybeJob = getPeonJob(taskId.getK8sJobName());
+ if (maybeJob.isPresent()) {
+ jobSeenInCache = true;
+ Job job = maybeJob.get();
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] found in cache and is still running",
taskId.getK8sJobName());
+ }
+ } else if (jobSeenInCache) {
+ // Job was in cache before, but now it's gone - it was deleted and
will never complete.
+ log.warn("K8s Job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ // We wake up every informer resync period to avoid event notifier
misses.
+ Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(),
TimeUnit.MILLISECONDS);
+
+ // Immediately set up to watch for the next change in case we need to
wait again
+ jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+ log.debug("Received job[%s] change notification",
taskId.getK8sJobName());
+ jobSeenInCache = true;
+
+ if (job == null) {
+ log.warn("K8s job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] is still running", taskId.getK8sJobName());
+ }
+ }
+ catch (TimeoutException e) {
+ // A timeout here is not a problem, it forces us to loop around and
check the cache again.
+ // This prevents the case where we miss a notification and wait
forever.
+ log.debug("Timeout waiting for job change notification for job[%s]. If
full job timeout has not been reached, the job completion wait will continue",
taskId.getK8sJobName());
+ }
+ catch (InterruptedException e) {
+ throw DruidException.defensive(e, "Interrupted waiting for job change
notification for job[%s]", taskId.getK8sJobName());
+ }
+ catch (Throwable e) {
+ log.warn("Exception[%s] waiting for job change notification for
job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(),
e.getMessage());
+ }
+ } while ((System.currentTimeMillis() - startTime < timeoutMs) &&
(jobSeenInCache || System.currentTimeMillis() < jobMustBeSeenBy));
Review Comment:
Consider using a Druid `Stopwatch` instead.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java:
##########
@@ -19,15 +19,30 @@
package org.apache.druid.k8s.overlord.common;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
// Wraps all kubernetes api calls, to ensure you open and close connections
properly
public interface KubernetesClientApi
{
<T> T executeRequest(KubernetesExecutor<T> executor) throws
KubernetesResourceNotFoundException;
+ <T> T executePodCacheRequest(KubernetesInformerExecutor<T, Pod> executor);
+
+ <T> T executeJobCacheRequest(KubernetesInformerExecutor<T, Job> executor);
+
// use only when handling streams of data, example if you want to pass
around an input stream from a pod,
// then you would call this instead of executeRequest as you would want to
keep the connection open until you
// are done with the stream. Callers responsibility to clean up when using
this method
KubernetesClient getClient();
+
+ SharedIndexInformer<Pod> getPodInformer();
+
+ SharedIndexInformer<Job> getJobInformer();
+
+ long getInformerResyncPeriodMillis();
+
+ KubernetesResourceEventNotifier getEventNotifier();
Review Comment:
Instead of returning the event notifier itself, we should expose the wait
methods, `waitForJobChange()`, `waitForPodChange()`.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java:
##########
@@ -53,4 +115,176 @@ public KubernetesClient getClient()
{
return this.kubernetesClient;
}
+
+ @Override
+ public SharedIndexInformer<Pod> getPodInformer()
+ {
+ if (podInformer == null) {
+ throw DruidException.defensive("Pod informer is not initialized, caching
is disabled");
+ }
+ return podInformer;
+ }
+
+ @Override
+ public SharedIndexInformer<Job> getJobInformer()
+ {
+ if (jobInformer == null) {
+ throw DruidException.defensive("Job informer is not initialized, caching
is disabled");
+ }
+ return jobInformer;
+ }
+
+ @Override
+ public KubernetesResourceEventNotifier getEventNotifier()
+ {
+ if (eventNotifier == null) {
+ throw DruidException.defensive("Event notifier is not initialized,
caching is disabled");
+ }
+ return eventNotifier;
+ }
+
+ /**
+ * Sets up a shared informer to watch and cache Pod resources in the
specified namespace.
+ * <p>
+ * Registers event handlers for pod add/update/delete events and creates a
custom index by job-name
+ * for efficient pod lookup by job.
+ * </p>
+ */
+ private SharedIndexInformer<Pod> setupPodInformer(String namespace)
+ {
+ SharedIndexInformer<Pod> podInformer =
+ kubernetesClient.pods()
+ .inNamespace(namespace)
+ .withLabel(DruidK8sConstants.LABEL_KEY)
+ .inform(
+ new ResourceEventHandler<>()
+ {
+ @Override
+ public void onAdd(Pod pod)
+ {
+ log.debug("Pod[%s] got added",
pod.getMetadata().getName());
+ notifyPodChange(pod);
+ }
+
+ @Override
+ public void onUpdate(Pod oldPod, Pod newPod)
+ {
+ log.debug("Pod[%s] got updated",
oldPod.getMetadata().getName());
+ notifyPodChange(newPod);
+ }
+
+ @Override
+ public void onDelete(Pod pod, boolean
deletedFinalStateUnknown)
+ {
+ log.debug("Pod[%s] got deleted",
pod.getMetadata().getName());
+ notifyPodChange(pod);
+ }
+ }, informerResyncPeriodMillis
+ );
+
+ Function<Pod, List<String>> jobNameIndexer = pod -> {
+ if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) {
+ String jobName = pod.getMetadata().getLabels().get("job-name");
+ if (jobName != null) {
+ return Collections.singletonList(jobName);
+ }
+ }
+ return Collections.emptyList();
+ };
+
+ Map<String, Function<Pod, List<String>>> customPodIndexers = new
HashMap<>();
+ customPodIndexers.put(JOB_NAME_INDEX, jobNameIndexer);
+
+ podInformer.addIndexers(customPodIndexers);
+ return podInformer;
+ }
+
+ /**
+ * Sets up a shared informer to watch and cache Job resources in the
specified namespace.
+ * <p>
+ * Registers event handlers for job add/update/delete events and creates
custom indexes by job-name
+ * and overlord-namespace for efficient job lookup and filtering.
+ * </p>
+ */
+ private SharedIndexInformer<Job> setupJobInformer(String namespace)
+ {
+ SharedIndexInformer<Job> jobInformer =
+ kubernetesClient.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withLabel(DruidK8sConstants.LABEL_KEY)
+ .inform(
+ new ResourceEventHandler<>()
Review Comment:
The `ResourceEventHandler` impls seem to take up too many lines.
Let's create a concrete impl, say `InformerEventHandler<T>` which just
accepts a single lambda `BiConsumer<T, EventType>` and invokes that lambda in
all of `onAdd`, `onUpdate`, `onDelete` with the correct event type.
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java:
##########
@@ -408,6 +405,17 @@ private void waitForSegmentsToBeQueryable(int numSegments)
);
}
+ /**
+ * Gets the count of tasks with the given status for the specified
datasource.
+ */
+ private int getTaskCount(String status, String dataSource)
Review Comment:
Consider moving this utility method to `EmbeddedClusterApis` itself so that
it may be used by other tests too.
##########
extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java:
##########
@@ -57,8 +58,14 @@ public void setup()
.withCapacity(1)
.build();
taskLogs = new NoopTaskLogs();
+
+ Config config = new ConfigBuilder().build();
+ config.setAdditionalProperty(
+ DruidKubernetesClient.ENABLE_INFORMERS_KEY,
+ kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers());
Review Comment:
Nit: style
```suggestion
kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()
);
```
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java:
##########
@@ -243,7 +242,7 @@ public void test_runIndexParallelTask_andCompactData()
.dynamicPartitionWithMaxRows(5000)
.withId(compactTaskId);
cluster.callApi().onLeaderOverlord(o -> o.runTask(compactTaskId,
compactionTask));
- cluster.callApi().waitForTaskToSucceed(taskId,
eventCollector.latchableEmitter());
+ cluster.callApi().waitForTaskToSucceed(compactTaskId,
eventCollector.latchableEmitter());
Review Comment:
Good catch!
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A KubernetesPeonClient implementation that uses shared informers to read
Job and Pod state from a local cache.
+ * <p>
+ * This implementation greatly reduces load on the Kubernetes API server by
centralizing watches and allowing
+ * tasks to query cached resource state instead of making per-task API calls.
Mutable operations (job creation,
+ * deletion) still contact the API server directly.
+ * </p>
+ */
+public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient
+{
+ protected static final EmittingLogger log = new
EmittingLogger(CachingKubernetesPeonClient.class);
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ String overlordNamespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, overlordNamespace, debugJobs, emitter);
+ }
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, "", debugJobs, emitter);
+ }
+
+ @Override
+ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong,
TimeUnit unit)
+ {
+ long timeoutMs = unit.toMillis(howLong);
+ long startTime = System.currentTimeMillis();
+
+ // Give the informer 2 resync periods to see the job. if it isn't seen by
then, we assume the job was canceled.
+ // This is to prevent us from waiting for entire max job runtime on a job
that was canceled before it even started.
+ long jobMustBeSeenBy = startTime +
(clientApi.getInformerResyncPeriodMillis() * 2);
+ boolean jobSeenInCache = false;
+
+ // Set up to watch for job changes
+ CompletableFuture<Job> jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+
+ // We will loop until the full timeout is reached if the job is seen in
cache. If the job does not show up in the cache we will exit earlier.
+ // In this loop we first check the cache to see if our job is there and
complete. This avoids missing notifications that happened before we set up the
watch.
+ // If the job is not complete we wait for a notification of a job change
or a timeout.
+ // If it is a timeout, we loop back to check the cache again.
+ // If it is a job change notification, we check the job state and exit if
complete, or loop again if still running.
+ do {
+ try {
+ Optional<Job> maybeJob = getPeonJob(taskId.getK8sJobName());
+ if (maybeJob.isPresent()) {
+ jobSeenInCache = true;
+ Job job = maybeJob.get();
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] found in cache and is still running",
taskId.getK8sJobName());
+ }
+ } else if (jobSeenInCache) {
+ // Job was in cache before, but now it's gone - it was deleted and
will never complete.
+ log.warn("K8s Job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ // We wake up every informer resync period to avoid event notifier
misses.
+ Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(),
TimeUnit.MILLISECONDS);
+
+ // Immediately set up to watch for the next change in case we need to
wait again
+ jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+ log.debug("Received job[%s] change notification",
taskId.getK8sJobName());
+ jobSeenInCache = true;
+
+ if (job == null) {
+ log.warn("K8s job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] is still running", taskId.getK8sJobName());
+ }
+ }
+ catch (TimeoutException e) {
+ // A timeout here is not a problem, it forces us to loop around and
check the cache again.
+ // This prevents the case where we miss a notification and wait
forever.
+ log.debug("Timeout waiting for job change notification for job[%s]. If
full job timeout has not been reached, the job completion wait will continue",
taskId.getK8sJobName());
Review Comment:
```suggestion
log.debug("Timeout waiting for change notification of job[%s].
Waiting until full job timeout.", taskId.getK8sJobName());
```
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java:
##########
@@ -80,6 +80,13 @@ public class KubernetesTaskRunnerConfig
// disable using http / https proxy environment variables
private boolean disableClientProxy;
+ @JsonProperty
+ // enable using kubernetes informer cache for peon client operations
+ private boolean enableKubernetesClientSharedInformers = false;
Review Comment:
Maybe use a simpler name
```suggestion
private boolean useK8sSharedInformers = false;
```
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A KubernetesPeonClient implementation that uses shared informers to read
Job and Pod state from a local cache.
+ * <p>
+ * This implementation greatly reduces load on the Kubernetes API server by
centralizing watches and allowing
+ * tasks to query cached resource state instead of making per-task API calls.
Mutable operations (job creation,
+ * deletion) still contact the API server directly.
+ * </p>
+ */
+public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient
+{
+ protected static final EmittingLogger log = new
EmittingLogger(CachingKubernetesPeonClient.class);
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ String overlordNamespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, overlordNamespace, debugJobs, emitter);
+ }
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, "", debugJobs, emitter);
+ }
+
+ @Override
+ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong,
TimeUnit unit)
+ {
+ long timeoutMs = unit.toMillis(howLong);
+ long startTime = System.currentTimeMillis();
+
+ // Give the informer 2 resync periods to see the job. if it isn't seen by
then, we assume the job was canceled.
+ // This is to prevent us from waiting for entire max job runtime on a job
that was canceled before it even started.
+ long jobMustBeSeenBy = startTime +
(clientApi.getInformerResyncPeriodMillis() * 2);
+ boolean jobSeenInCache = false;
+
+ // Set up to watch for job changes
+ CompletableFuture<Job> jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+
+ // We will loop until the full timeout is reached if the job is seen in
cache. If the job does not show up in the cache we will exit earlier.
+ // In this loop we first check the cache to see if our job is there and
complete. This avoids missing notifications that happened before we set up the
watch.
+ // If the job is not complete we wait for a notification of a job change
or a timeout.
+ // If it is a timeout, we loop back to check the cache again.
+ // If it is a job change notification, we check the job state and exit if
complete, or loop again if still running.
+ do {
+ try {
+ Optional<Job> maybeJob = getPeonJob(taskId.getK8sJobName());
+ if (maybeJob.isPresent()) {
+ jobSeenInCache = true;
+ Job job = maybeJob.get();
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] found in cache and is still running",
taskId.getK8sJobName());
+ }
+ } else if (jobSeenInCache) {
+ // Job was in cache before, but now it's gone - it was deleted and
will never complete.
+ log.warn("K8s Job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ // We wake up every informer resync period to avoid event notifier
misses.
+ Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(),
TimeUnit.MILLISECONDS);
+
+ // Immediately set up to watch for the next change in case we need to
wait again
+ jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+ log.debug("Received job[%s] change notification",
taskId.getK8sJobName());
+ jobSeenInCache = true;
+
+ if (job == null) {
+ log.warn("K8s job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] is still running", taskId.getK8sJobName());
+ }
+ }
+ catch (TimeoutException e) {
+ // A timeout here is not a problem, it forces us to loop around and
check the cache again.
+ // This prevents the case where we miss a notification and wait
forever.
+ log.debug("Timeout waiting for job change notification for job[%s]. If
full job timeout has not been reached, the job completion wait will continue",
taskId.getK8sJobName());
+ }
+ catch (InterruptedException e) {
+ throw DruidException.defensive(e, "Interrupted waiting for job change
notification for job[%s]", taskId.getK8sJobName());
+ }
+ catch (Throwable e) {
+ log.warn("Exception[%s] waiting for job change notification for
job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(),
e.getMessage());
Review Comment:
```suggestion
log.noStackTrace().warn(e, "Exception while waiting for change
notification of job[%s]", taskId.getK8sJobName());
```
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java:
##########
@@ -19,21 +19,64 @@
package org.apache.druid.k8s.overlord.common;
+import com.google.common.base.Preconditions;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.Config;
import io.fabric8.kubernetes.client.KubernetesClient;
import io.fabric8.kubernetes.client.KubernetesClientBuilder;
+import io.fabric8.kubernetes.client.informers.ResourceEventHandler;
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.emitter.EmittingLogger;
import
org.apache.druid.k8s.overlord.common.httpclient.DruidKubernetesHttpClientFactory;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
public class DruidKubernetesClient implements KubernetesClientApi
Review Comment:
All the changes in this class (and the corresponding APIs in
`KubernetesClientApi`) pertain only to the caching implementation.
I would suggest skipping the changes made to `KubernetesClientApi` and
`DruidKubernetesClient` and move all the new logic into a separate class
altogether and use it in `CachingKubernetesPeonClient` directly. The
non-caching impl should not be able to invoke these methods at all (rather than
initializing `podInformer` etc to null and throw exceptions if invoked).
The new class can be a wrapper on `DruidKubernetesClient` and may use
`DruidKubernetesClient.getClient()`.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
+
+@FunctionalInterface
+public interface KubernetesInformerExecutor<T, R>
Review Comment:
Maybe rename to `SharedInformerCacheReadRequest` to be more specific?
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java:
##########
@@ -19,15 +19,30 @@
package org.apache.druid.k8s.overlord.common;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
// Wraps all kubernetes api calls, to ensure you open and close connections
properly
public interface KubernetesClientApi
{
<T> T executeRequest(KubernetesExecutor<T> executor) throws
KubernetesResourceNotFoundException;
+ <T> T executePodCacheRequest(KubernetesInformerExecutor<T, Pod> executor);
+
+ <T> T executeJobCacheRequest(KubernetesInformerExecutor<T, Job> executor);
+
// use only when handling streams of data, example if you want to pass
around an input stream from a pod,
// then you would call this instead of executeRequest as you would want to
keep the connection open until you
// are done with the stream. Callers responsibility to clean up when using
this method
KubernetesClient getClient();
+
+ SharedIndexInformer<Pod> getPodInformer();
+
+ SharedIndexInformer<Job> getJobInformer();
Review Comment:
I guess these methods don't need to be exposed, as they are not being used
anywhere.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A KubernetesPeonClient implementation that uses shared informers to read
Job and Pod state from a local cache.
+ * <p>
+ * This implementation greatly reduces load on the Kubernetes API server by
centralizing watches and allowing
+ * tasks to query cached resource state instead of making per-task API calls.
Mutable operations (job creation,
+ * deletion) still contact the API server directly.
+ * </p>
+ */
+public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient
+{
+ protected static final EmittingLogger log = new
EmittingLogger(CachingKubernetesPeonClient.class);
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ String overlordNamespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, overlordNamespace, debugJobs, emitter);
+ }
+
+ public CachingKubernetesPeonClient(
+ KubernetesClientApi clientApi,
+ String namespace,
+ boolean debugJobs,
+ ServiceEmitter emitter
+ )
+ {
+ super(clientApi, namespace, "", debugJobs, emitter);
+ }
+
+ @Override
+ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong,
TimeUnit unit)
+ {
+ long timeoutMs = unit.toMillis(howLong);
+ long startTime = System.currentTimeMillis();
+
+ // Give the informer 2 resync periods to see the job. if it isn't seen by
then, we assume the job was canceled.
+ // This is to prevent us from waiting for entire max job runtime on a job
that was canceled before it even started.
+ long jobMustBeSeenBy = startTime +
(clientApi.getInformerResyncPeriodMillis() * 2);
+ boolean jobSeenInCache = false;
+
+ // Set up to watch for job changes
+ CompletableFuture<Job> jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
+
+ // We will loop until the full timeout is reached if the job is seen in
cache. If the job does not show up in the cache we will exit earlier.
+ // In this loop we first check the cache to see if our job is there and
complete. This avoids missing notifications that happened before we set up the
watch.
+ // If the job is not complete we wait for a notification of a job change
or a timeout.
+ // If it is a timeout, we loop back to check the cache again.
+ // If it is a job change notification, we check the job state and exit if
complete, or loop again if still running.
+ do {
+ try {
+ Optional<Job> maybeJob = getPeonJob(taskId.getK8sJobName());
+ if (maybeJob.isPresent()) {
+ jobSeenInCache = true;
+ Job job = maybeJob.get();
+ JobResponse currentResponse = determineJobResponse(job);
+ if (currentResponse.getPhase() != PeonPhase.RUNNING) {
+ return currentResponse;
+ } else {
+ log.debug("K8s job[%s] found in cache and is still running",
taskId.getK8sJobName());
+ }
+ } else if (jobSeenInCache) {
+ // Job was in cache before, but now it's gone - it was deleted and
will never complete.
+ log.warn("K8s Job[%s] was not found. It can happen if the task was
canceled", taskId.getK8sJobName());
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+
+ // We wake up every informer resync period to avoid event notifier
misses.
+ Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(),
TimeUnit.MILLISECONDS);
+
+ // Immediately set up to watch for the next change in case we need to
wait again
+ jobFuture =
clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName());
Review Comment:
Can there be a race between these two lines causing us to miss out on
notifications?
How about we create the next future before invoking `.get()` on the previous
one?
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java:
##########
@@ -19,15 +19,30 @@
package org.apache.druid.k8s.overlord.common;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
// Wraps all kubernetes api calls, to ensure you open and close connections
properly
public interface KubernetesClientApi
{
<T> T executeRequest(KubernetesExecutor<T> executor) throws
KubernetesResourceNotFoundException;
+ <T> T executePodCacheRequest(KubernetesInformerExecutor<T, Pod> executor);
+
+ <T> T executeJobCacheRequest(KubernetesInformerExecutor<T, Job> executor);
Review Comment:
Nit: Should we simplify these names to `readPodCache` / `getFromPodCache` or
similar?
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java:
##########
Review Comment:
Yes, let's postpone the refactors/optimizations for follow up PRs.
It would also be helpful to get rid of `AbstractKubernetesClient` and retain
`KubernetesPeonClient` same as before. The new caching-based impl may extend
`KubernetesPeonClient` and override whatever is needed.
Let's make only the necessary minimal changes to `KubernetesPeonClient` in
this patch.
This would help focus this PR only on the new stuff, avoid copy paste errors
and greatly simplify reviews.
I was trying it out here https://github.com/kfaraz/druid/pull/356. You may
cherrypick the last commits in that branch if you find them to be useful.
##########
extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import io.fabric8.kubernetes.client.informers.SharedIndexInformer;
+
+@FunctionalInterface
+public interface KubernetesInformerExecutor<T, R>
+{
+ T executeRequest(SharedIndexInformer<R> informer);
Review Comment:
Should this function be applied on the `Indexer` instead of the
`SharedIndexInformer`?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]